Skip to content

Commit

Permalink
Support libpython.so to profile python docker containers
Browse files Browse the repository at this point in the history
commit_hash:f47e1c6ad803eaf2fdd38d745bb2776d484fd703
  • Loading branch information
pashagoose committed Feb 19, 2025
1 parent 11ef36d commit 6b56359
Show file tree
Hide file tree
Showing 9 changed files with 104 additions and 56 deletions.
3 changes: 0 additions & 3 deletions perforator/agent/collector/pkg/dso/bpf/python/manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,6 @@ func (m *BPFManager) Add(id uint64, conf *pythonpreprocessing.PythonConfig) erro
}

pythonUnwindConfig := python_agent.ParsePythonUnwinderConfig(conf)
if pythonUnwindConfig == nil {
return nil
}

m.mutex.Lock()
defer m.mutex.Unlock()
Expand Down
26 changes: 24 additions & 2 deletions perforator/agent/collector/pkg/dso/map.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@ import (
"github.com/yandex/perforator/perforator/agent/collector/pkg/binary"
bpf "github.com/yandex/perforator/perforator/agent/collector/pkg/dso/bpf/binary"
"github.com/yandex/perforator/perforator/agent/collector/pkg/dso/parser"
python_agent "github.com/yandex/perforator/perforator/internal/linguist/python/agent"
"github.com/yandex/perforator/perforator/internal/unwinder"
"github.com/yandex/perforator/perforator/pkg/xelf"
"github.com/yandex/perforator/perforator/pkg/xlog"
)
Expand All @@ -26,6 +28,9 @@ type dso struct {
// Unique ID of the DSO. It is used by eBPF.
ID uint64

// Type of the interpreter.
InterpreterType unwinder.InterpreterType

// Build info of the binary.
buildInfo *xelf.BuildInfo

Expand Down Expand Up @@ -201,8 +206,9 @@ func (d *Registry) register(ctx context.Context, buildInfo *xelf.BuildInfo, file

item := d.trackingFetch(buildID, 10*time.Minute, func() *dso {
return &dso{
ID: d.nextid.Add(1) - 1,
buildInfo: buildInfo,
ID: d.nextid.Add(1) - 1,
buildInfo: buildInfo,
InterpreterType: unwinder.InterpreterTypeNone,
}
})

Expand Down Expand Up @@ -297,6 +303,22 @@ func (d *Registry) populateDSO(ctx context.Context, dso *dso, f *os.File) {
return
}

if analysis.PythonConfig != nil && !python_agent.IsVersionSupported(analysis.PythonConfig.Version) {
if analysis.PythonConfig.Version != nil {
d.l.Debug(
ctx,
"Python version is not supported, removing python config from binary analysis",
log.String("buildid", buildID),
log.Any("version", analysis.PythonConfig.Version),
)
}
analysis.PythonConfig = nil
}

if analysis.PythonConfig != nil {
dso.InterpreterType = unwinder.InterpreterTypePython
}

dso.bpfAllocation, err = d.bpfBinaryManager.Add(buildID, dso.ID, analysis)
if err != nil {
d.l.Error(
Expand Down
18 changes: 18 additions & 0 deletions perforator/agent/collector/pkg/process/map.go
Original file line number Diff line number Diff line change
Expand Up @@ -605,6 +605,23 @@ func (a *processAnalyzer) registerMapping(m *dso.Mapping) {
a.exemappings = append(a.exemappings, m)
}

func (a *processAnalyzer) findInterpreterBinary(mappings []*dso.Mapping) (binary unwinder.InterpreterBinary, baseAddress uint64) {
for _, m := range mappings {
if m.DSO == nil {
continue
}

if m.DSO.InterpreterType != unwinder.InterpreterTypeNone {
return unwinder.InterpreterBinary{
Id: unwinder.BinaryId(m.DSO.ID),
Type: m.DSO.InterpreterType,
}, m.BaseAddress
}
}

return unwinder.InterpreterBinary{}, 0
}

func (a *processAnalyzer) storeBPFMaps(ctx context.Context) error {
sort.Slice(a.exemappings, func(i, j int) bool {
return a.exemappings[i].Begin < a.exemappings[j].Begin
Expand All @@ -620,6 +637,7 @@ func (a *processAnalyzer) storeBPFMaps(ctx context.Context) error {
} else {
pi.MainBinaryId = unwinder.BinaryId(math.MaxUint64)
}
pi.InterpreterBinary, pi.InterpreterBinaryStartAddress = a.findInterpreterBinary(a.exemappings)

a.log.Debug(ctx, "Put process info", log.Any("info", pi))
err := a.reg.bpf.AddProcess(a.proc.id, &pi)
Expand Down
7 changes: 0 additions & 7 deletions perforator/agent/collector/progs/unwinder/core.h
Original file line number Diff line number Diff line change
Expand Up @@ -86,17 +86,10 @@ struct pid_namespace {

////////////////////////////////////////////////////////////////////////////////

struct mm_struct {
unsigned long start_code;
};

////////////////////////////////////////////////////////////////////////////////

struct task_struct {
void* stack;
unsigned int flags;
struct task_struct* group_leader;
struct mm_struct* mm;
u64 real_start_time;
char comm[TASK_COMM_LEN];
struct css_set* cgroups;
Expand Down
16 changes: 16 additions & 0 deletions perforator/agent/collector/progs/unwinder/process.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,23 @@ enum unwind_type : u8 {
UNWIND_TYPE_DWARF = 2,
};

enum interpreter_type : u8 {
INTERPRETER_TYPE_NONE = 0,
INTERPRETER_TYPE_PYTHON = 1,
};

struct interpreter_binary {
binary_id id;
enum interpreter_type type;
};

struct process_info {
enum unwind_type unwind_type;
binary_id main_binary_id;

// Interpreter binary ID may be different from main binary ID.
// For example, if CPython is dynamically linked
// into the main binary.
struct interpreter_binary interpreter_binary;
u64 interpreter_binary_start_address;
};
45 changes: 22 additions & 23 deletions perforator/agent/collector/progs/unwinder/py_threads.h
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,10 @@ struct python_thread_state_offsets {
BPF_MAP(python_thread_id_py_thread_state, BPF_MAP_TYPE_LRU_HASH, u32, void*, MAX_PYTHON_THREADS);

static ALWAYS_INLINE void* python_read_py_thread_state_ptr_from_tls(u64 offset) {
if (offset == 0) {
return NULL;
}

struct task_struct* task = (void*)bpf_get_current_task();

unsigned long fsbase = BPF_CORE_READ(task, thread.fsbase);
Expand All @@ -53,11 +57,11 @@ static ALWAYS_INLINE void* python_read_py_thread_state_ptr_from_tls(u64 offset)
static ALWAYS_INLINE void* python_get_py_thread_state_from_cache(u32 native_thread_id) {
void** py_thread_state_ptr = bpf_map_lookup_elem(&python_thread_id_py_thread_state, &native_thread_id);
if (py_thread_state_ptr == NULL) {
BPF_TRACE("python: failed to find Pythread_state for native thread ID %u", native_thread_id);
BPF_TRACE("python: failed to find PyThreadState for native thread ID %u", native_thread_id);
return NULL;
}

BPF_TRACE("python: successfully retrieved Pythread_state for native thread ID %u", native_thread_id);
BPF_TRACE("python: successfully retrieved PyThreadState for native thread ID %u", native_thread_id);
return *py_thread_state_ptr;
}

Expand Down Expand Up @@ -101,39 +105,32 @@ static NOINLINE void python_upsert_thread_state(void* py_thread_state, struct py
if (err != 0) {
BPF_TRACE("python: failed to update BPF map with native thread ID %u: %d", native_thread_id, err);
}
}

// Bypass ASLR
static ALWAYS_INLINE void* python_get_global_runtime_address(u64 py_runtime_relative_address) {
struct task_struct* task = (void*)bpf_get_current_task();
unsigned long base_addr = BPF_CORE_READ(task, mm, start_code);
return (void*) (base_addr + py_runtime_relative_address);
BPF_TRACE("python: successfully upserted PyThreadState %p for native thread ID %u", (void*) py_thread_state, native_thread_id);
}

static ALWAYS_INLINE void* python_retrieve_main_interpreterstate(u64 py_runtime_relative_address, struct python_runtime_state_offsets* runtime_state_offsets) {
if (py_runtime_relative_address == 0 || runtime_state_offsets == NULL) {
static ALWAYS_INLINE void* python_retrieve_main_interpreterstate(void* py_runtime_ptr, struct python_runtime_state_offsets* runtime_state_offsets) {
if (py_runtime_ptr == NULL || runtime_state_offsets == NULL) {
return NULL;
}

void* py_runtime_address = python_get_global_runtime_address(py_runtime_relative_address);

void* main_interpreter_state = NULL;
long err = bpf_probe_read_user(
&main_interpreter_state,
sizeof(void*),
py_runtime_address + runtime_state_offsets->py_interpreters_main_offset
py_runtime_ptr + runtime_state_offsets->py_interpreters_main_offset
);
if (err != 0) {
BPF_TRACE("python: failed to read main PyInterpreterState: %d", err);
return NULL;
}

if (main_interpreter_state == NULL) {
BPF_TRACE("python: main *PyInterpreterState is NULL");
return NULL;
}

BPF_TRACE("python: successfully retrieved main *PyInterpreterState");
BPF_TRACE("python: successfully retrieved main PyInterpreterState %p", main_interpreter_state);

return main_interpreter_state;
}

Expand All @@ -153,26 +150,28 @@ static ALWAYS_INLINE void* python_retrieve_thread_state_from_interpreterstate(vo
return NULL;
}

BPF_TRACE("python: successfully retrieved head *Pythread_state from *PyInterpreterState");
BPF_TRACE("python: successfully retrieved head *PyThreadState from *PyInterpreterState");
return head_thread_state;
}

static ALWAYS_INLINE void* python_get_head_thread_state(
u64 py_runtime_relative_address,
u64 py_runtime_address,
struct python_runtime_state_offsets* runtime_state_offsets,
struct python_interpreter_state_offsets* interpreter_state_offsets
) {
if (py_runtime_relative_address == 0 || runtime_state_offsets == NULL || interpreter_state_offsets == NULL) {
if (py_runtime_address == 0 || runtime_state_offsets == NULL || interpreter_state_offsets == NULL) {
return NULL;
}

void* main_interpreter_state = python_retrieve_main_interpreterstate(py_runtime_relative_address, runtime_state_offsets);
void* main_interpreter_state = python_retrieve_main_interpreterstate((void*) py_runtime_address, runtime_state_offsets);
void* head_thread_state = python_retrieve_thread_state_from_interpreterstate(main_interpreter_state, interpreter_state_offsets);

if (head_thread_state == NULL) {
BPF_TRACE("python: head *Pythread_state from *PyInterpreterState is NULL");
BPF_TRACE("python: head *PyThreadState from *PyInterpreterState is NULL");
}

BPF_TRACE("python: successfully retrieved head *PyThreadState from *PyInterpreterState");

return head_thread_state;
}

Expand Down Expand Up @@ -226,7 +225,7 @@ static ALWAYS_INLINE void python_fill_threads_cache(void* py_thread_state, struc

static ALWAYS_INLINE void* python_get_thread_state_and_update_cache(
u64 py_thread_state_tls_offset,
u64 py_runtime_relative_address,
u64 py_runtime_address,
struct python_runtime_state_offsets* runtime_state_offsets,
struct python_interpreter_state_offsets* interpreter_state_offsets,
struct python_thread_state_offsets* thread_state_offsets
Expand All @@ -240,7 +239,7 @@ static ALWAYS_INLINE void* python_get_thread_state_and_update_cache(

void *fill_cache_thread_state = current_thread_state;
if (fill_cache_thread_state == NULL) {
fill_cache_thread_state = python_get_head_thread_state(py_runtime_relative_address, runtime_state_offsets, interpreter_state_offsets);
fill_cache_thread_state = python_get_head_thread_state(py_runtime_address, runtime_state_offsets, interpreter_state_offsets);
}

python_fill_threads_cache(fill_cache_thread_state, thread_state_offsets);
Expand All @@ -250,7 +249,7 @@ static ALWAYS_INLINE void* python_get_thread_state_and_update_cache(
}

if (current_thread_state == NULL) {
BPF_TRACE("python: failed to retrieve Pythread_state from both TLS and cache for thread");
BPF_TRACE("python: failed to retrieve PyThreadState from both TLS and cache for thread");
}

return current_thread_state;
Expand Down
14 changes: 9 additions & 5 deletions perforator/agent/collector/progs/unwinder/python.h
Original file line number Diff line number Diff line change
Expand Up @@ -406,29 +406,33 @@ static ALWAYS_INLINE void python_collect_stack(
return;
}

// TODO(@pashaguskov): support libpython.so
binary_id id = proc_info->main_binary_id;
if (proc_info->interpreter_binary.type != INTERPRETER_TYPE_PYTHON) {
return;
}

binary_id id = proc_info->interpreter_binary.id;
struct python_config* config = bpf_map_lookup_elem(&python_storage, &id);
if (config == NULL) {
// sanity check, should be not NULL because of previous check for python interpreter type
return;
}

metric_increment(METRIC_PYTHON_PROCESSED_STACKS_COUNT);

void* py_thread_state_addr = python_get_thread_state_and_update_cache(
config->py_thread_state_tls_offset,
config->py_runtime_relative_address,
proc_info->interpreter_binary_start_address + config->py_runtime_relative_address,
&config->offsets.py_runtime_state_offsets,
&config->offsets.py_interpreter_state_offsets,
&config->offsets.py_thread_state_offsets
);
if (py_thread_state_addr == NULL) {
metric_increment(METRIC_PYTHON_TLS_THREAD_STATE_NULL);
BPF_TRACE("python: read NULL *Pythread_state");
BPF_TRACE("python: read NULL *PyThreadState");
return;
}

BPF_TRACE("python: Successfully extracted Pythread_state addr %p", py_thread_state_addr);
BPF_TRACE("python: Successfully extracted PyThreadState addr %p", py_thread_state_addr);

void* py_interpreter_frame = python_read_current_frame_from_thread_state(config, py_thread_state_addr);
if (py_interpreter_frame == NULL) {
Expand Down
10 changes: 4 additions & 6 deletions perforator/agent/preprocessing/lib/analyze.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -43,16 +43,14 @@ NPerforator::NBinaryProcessing::NPython::PythonConfig BuildPythonConfig(llvm::ob
conf.MutableVersion()->SetMicro(version->Version.MicroVersion);

auto threadStateTLSOffset = analyzer.ParseTLSPyThreadState();
if (!threadStateTLSOffset) {
return conf;
if (threadStateTLSOffset) {
conf.SetPyThreadStateTLSOffset(*threadStateTLSOffset);
}
conf.SetPyThreadStateTLSOffset(*threadStateTLSOffset);

auto pyRuntimeAddress = analyzer.ParsePyRuntimeAddress();
if (!pyRuntimeAddress) {
return conf;
if (pyRuntimeAddress) {
conf.SetRelativePyRuntimeAddress(*pyRuntimeAddress);
}
conf.SetRelativePyRuntimeAddress(*pyRuntimeAddress);

return conf;
}
Expand Down
21 changes: 11 additions & 10 deletions perforator/internal/linguist/python/agent/python.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,23 +53,24 @@ func PythonInternalsOffsetsByVersion(version *python.PythonVersion) (*unwinder.P
}

func IsVersionSupported(version *python.PythonVersion) bool {
if version == nil {
return false
}

return version.Major == 3 && (version.Minor == 12 || version.Minor == 13)
}

func encodeVersion(version *python.PythonVersion) uint32 {
return version.Micro + (version.Minor << 8) + (version.Major)<<16
}

// Only supported python version config must be passed here.
func ParsePythonUnwinderConfig(conf *python.PythonConfig) *unwinder.PythonConfig {
if conf != nil && conf.PyThreadStateTLSOffset < 0 && conf.Version != nil && IsVersionSupported(conf.Version) {
offsets, _ := PythonInternalsOffsetsByVersion(conf.Version)
return &unwinder.PythonConfig{
Version: encodeVersion(conf.Version),
PyThreadStateTlsOffset: uint64(-conf.PyThreadStateTLSOffset),
PyRuntimeRelativeAddress: conf.RelativePyRuntimeAddress,
Offsets: *offsets,
}
offsets, _ := PythonInternalsOffsetsByVersion(conf.Version)
return &unwinder.PythonConfig{
Version: encodeVersion(conf.Version),
PyThreadStateTlsOffset: uint64(-conf.PyThreadStateTLSOffset),
PyRuntimeRelativeAddress: conf.RelativePyRuntimeAddress,
Offsets: *offsets,
}

return nil
}

0 comments on commit 6b56359

Please sign in to comment.