diff --git a/bpf/errors.h b/bpf/errors.h new file mode 100644 index 000000000..2d660de47 --- /dev/null +++ b/bpf/errors.h @@ -0,0 +1,27 @@ +#ifndef __ERRORS_H_ +#define __ERRORS_H_ + +#ifndef TASK_COMM_LEN +#define TASK_COMM_LEN 16 +#endif + +#ifndef ERR_MSG_LEN +#define ERR_MSG_LEN 32 +#endif + +#ifndef MAX_STACK_DEPTH +#define MAX_STACK_DEPTH 32 +#endif + +typedef __u64 stack_trace_t[MAX_STACK_DEPTH]; + +typedef struct error_event { + __u32 pid; + __u32 cpu_id; + char comm[TASK_COMM_LEN]; + __s32 ustack_sz; + stack_trace_t ustack; + u8 err_msg[ERR_MSG_LEN]; +} error_event; + +#endif /* __ERRORS_H_ */ diff --git a/bpf/go_nethttp.c b/bpf/go_nethttp.c index fb6524c55..578913bcd 100644 --- a/bpf/go_nethttp.c +++ b/bpf/go_nethttp.c @@ -25,6 +25,7 @@ #include "tracing.h" #include "hpack.h" #include "ringbuf.h" +#include "errors.h" typedef struct http_func_invocation { u64 start_monotime_ns; @@ -46,6 +47,13 @@ struct { __uint(max_entries, MAX_CONCURRENT_REQUESTS); } ongoing_http_client_requests SEC(".maps"); +struct { + __uint(type, BPF_MAP_TYPE_LRU_HASH); + __type(key, void *); // key: pointer to the request goroutine + __type(value, struct error_event); + __uint(max_entries, MAX_CONCURRENT_REQUESTS); +} last_error SEC(".maps"); + struct { __uint(type, BPF_MAP_TYPE_LRU_HASH); __type(key, void *); // key: pointer to the request goroutine @@ -198,6 +206,9 @@ int uprobe_ServeHTTPReturns(struct pt_regs *ctx) { resp_ptr = deref_resp_ptr; } + struct error_event *error = bpf_map_lookup_elem(&last_error, &goroutine_addr); + bpf_map_delete_elem(&last_error, &goroutine_addr); + http_request_trace *trace = bpf_ringbuf_reserve(&events, sizeof(http_request_trace), 0); if (!trace) { bpf_dbg_printk("can't reserve space in the ringbuffer"); @@ -208,6 +219,8 @@ int uprobe_ServeHTTPReturns(struct pt_regs *ctx) { trace->type = EVENT_HTTP_REQUEST; trace->start_monotime_ns = invocation->start_monotime_ns; trace->end_monotime_ns = bpf_ktime_get_ns(); + if (error) + trace->error = *error; goroutine_metadata *g_metadata = bpf_map_lookup_elem(&ongoing_goroutines, &goroutine_addr); if (g_metadata) { @@ -436,6 +449,51 @@ int uprobe_roundTripReturn(struct pt_regs *ctx) { return 0; } + +SEC("uprobe/error") +int uprobe_error(struct pt_regs *ctx) { + bpf_dbg_printk("=== uprobe/proc error return === "); + + void *goroutine_addr = GOROUTINE_PTR(ctx); + bpf_dbg_printk("goroutine_addr %lx", goroutine_addr); + + int pid = bpf_get_current_pid_tgid() >> 32; + int cpu_id = bpf_get_smp_processor_id(); + int BPF_F_USER_STACK = (1ULL << 8); + + struct error_event event = { + .pid = pid, + .cpu_id = cpu_id, + }; + + // Read the error message + void *msg_ptr = GO_PARAM1(ctx); + if (!read_go_str_n("error", msg_ptr, (u64)GO_PARAM2(ctx), &event.err_msg, sizeof(event.err_msg))) { + bpf_printk("can't read error message"); + return 0; + } + bpf_dbg_printk("error msg %llx, %llx", msg_ptr, GO_PARAM2(ctx)); + + if (bpf_get_current_comm(event.comm, sizeof(event.comm))) + event.comm[0] = 0; + + // Read the stack trace + event.ustack_sz = bpf_get_stack(ctx, event.ustack, sizeof(event.ustack), BPF_F_USER_STACK); + + // Get the caller of the error function and store it in the first slot of the stack + void *sp_caller = STACK_PTR(ctx); + u64 caller = 0; + bpf_probe_read(&caller, sizeof(u64), sp_caller); + bpf_dbg_printk("sp_caller %lx caller %lx", sp_caller, caller); + event.ustack[0] = caller; + + // Write event + if (bpf_map_update_elem(&last_error, &goroutine_addr, &event, BPF_ANY)) { + bpf_dbg_printk("can't update event error map element"); + } + + return 0; +} #ifndef NO_HEADER_PROPAGATION // Context propagation through HTTP headers SEC("uprobe/header_writeSubset") @@ -936,4 +994,4 @@ int uprobe_queryReturn(struct pt_regs *ctx) { bpf_dbg_printk("can't reserve space in the ringbuffer"); } return 0; -} \ No newline at end of file +} diff --git a/bpf/headers/utils.h b/bpf/headers/utils.h index 37413daef..217ef671d 100644 --- a/bpf/headers/utils.h +++ b/bpf/headers/utils.h @@ -34,7 +34,7 @@ // In x86, current goroutine is pointed by r14, according to // https://go.googlesource.com/go/+/refs/heads/dev.regabi/src/cmd/compile/internal-abi.md#amd64-architecture #define GOROUTINE_PTR(x) ((void*)(x)->r14) - +#define STACK_PTR(x) ((void*)(x)->sp) #elif defined(__TARGET_ARCH_arm64) #define GO_PARAM1(x) ((void*)((PT_REGS_ARM64 *)(x))->regs[0]) @@ -50,6 +50,7 @@ // In arm64, current goroutine is pointed by R28 according to // https://github.com/golang/go/blob/master/src/cmd/compile/abi-internal.md#arm64-architecture #define GOROUTINE_PTR(x) ((void*)((PT_REGS_ARM64 *)(x))->regs[28]) +#define STACK_PTR(x) ((void*)((PT_REGS_ARM64 *)(x))->regs[13]) #endif /*defined(__TARGET_ARCH_arm64)*/ diff --git a/bpf/http_trace.h b/bpf/http_trace.h index 8efe99318..a8e25af98 100644 --- a/bpf/http_trace.h +++ b/bpf/http_trace.h @@ -15,6 +15,7 @@ #include "pid_types.h" #include "utils.h" +#include "errors.h" #include "http_types.h" #define PATH_MAX_LEN 100 @@ -39,6 +40,7 @@ typedef struct http_request_trace_t { u16 status; connection_info_t conn __attribute__ ((aligned (8))); s64 content_length; + error_event error; tp_info_t tp; pid_info pid; diff --git a/pkg/beyla/config.go b/pkg/beyla/config.go index e60cb206a..3730c1438 100644 --- a/pkg/beyla/config.go +++ b/pkg/beyla/config.go @@ -73,11 +73,12 @@ var DefaultConfig = Config{ TTL: defaultMetricsTTL, }, Traces: otel.TracesConfig{ - Protocol: otel.ProtocolUnset, - TracesProtocol: otel.ProtocolUnset, - MaxQueueSize: 4096, - MaxExportBatchSize: 4096, - ReportersCacheLen: ReporterLRUSize, + Protocol: otel.ProtocolUnset, + TracesProtocol: otel.ProtocolUnset, + MaxQueueSize: 4096, + MaxExportBatchSize: 4096, + ReportersCacheLen: ReporterLRUSize, + ReportExceptionEvents: false, Instrumentations: []string{ instrumentations.InstrumentationALL, }, diff --git a/pkg/internal/discover/attacher.go b/pkg/internal/discover/attacher.go index 00d709188..c4d58c275 100644 --- a/pkg/internal/discover/attacher.go +++ b/pkg/internal/discover/attacher.go @@ -117,7 +117,7 @@ func (ta *TraceAttacher) getTracer(ie *Instrumentable) (*ebpf.ProcessTracer, boo } } else { tracerType = ebpf.Go - programs = filterNotFoundPrograms(newGoTracersGroup(ta.Cfg, ta.Metrics), ie.Offsets) + programs = filterNotFoundPrograms(newGoTracersGroup(ta.Cfg, ta.Metrics, ie.Offsets.SymTab), ie.Offsets) } case svc.InstrumentableNodejs: programs = ta.genericTracers() diff --git a/pkg/internal/discover/finder.go b/pkg/internal/discover/finder.go index 2834f677e..67239f836 100644 --- a/pkg/internal/discover/finder.go +++ b/pkg/internal/discover/finder.go @@ -2,6 +2,7 @@ package discover import ( "context" + "debug/gosym" "fmt" "github.com/mariomac/pipes/pipe" @@ -96,10 +97,10 @@ func (pf *ProcessFinder) Start() (<-chan *ebpf.ProcessTracer, <-chan *Instrument // auxiliary functions to instantiate the go and non-go tracers on diverse steps of the // discovery pipeline -func newGoTracersGroup(cfg *beyla.Config, metrics imetrics.Reporter) []ebpf.Tracer { +func newGoTracersGroup(cfg *beyla.Config, metrics imetrics.Reporter, symTab *gosym.Table) []ebpf.Tracer { // Each program is an eBPF source: net/http, grpc... return []ebpf.Tracer{ - nethttp.New(cfg, metrics), + nethttp.New(cfg, metrics, symTab), grpc.New(cfg, metrics), goruntime.New(cfg, metrics), sarama.New(cfg, metrics), diff --git a/pkg/internal/discover/typer.go b/pkg/internal/discover/typer.go index 91b9c2ccc..af96bc377 100644 --- a/pkg/internal/discover/typer.go +++ b/pkg/internal/discover/typer.go @@ -166,7 +166,7 @@ func (t *typer) inspectOffsets(execElf *exec.FileInfo) (*goexec.Offsets, bool, e t.log.Debug("skipping inspection for Go functions", "pid", execElf.Pid, "comm", execElf.CmdExePath) } else { t.log.Debug("inspecting", "pid", execElf.Pid, "comm", execElf.CmdExePath) - offsets, err := goexec.InspectOffsets(execElf, t.allGoFunctions) + offsets, err := goexec.InspectOffsets(&t.cfg.Traces, execElf, t.allGoFunctions) if err != nil { t.log.Debug("couldn't find go specific tracers", "error", err) return nil, false, err @@ -191,7 +191,7 @@ func isGoProxy(offsets *goexec.Offsets) bool { func (t *typer) loadAllGoFunctionNames() { uniqueFunctions := map[string]struct{}{} t.allGoFunctions = nil - for _, p := range newGoTracersGroup(t.cfg, t.metrics) { + for _, p := range newGoTracersGroup(t.cfg, t.metrics, nil) { for funcName := range p.GoProbes() { // avoid duplicating function names if _, ok := uniqueFunctions[funcName]; !ok { diff --git a/pkg/internal/ebpf/common/bpf_bpf.o b/pkg/internal/ebpf/common/bpf_bpf.o new file mode 100644 index 000000000..d175481fa Binary files /dev/null and b/pkg/internal/ebpf/common/bpf_bpf.o differ diff --git a/pkg/internal/ebpf/common/bpf_bpfel_arm64.go b/pkg/internal/ebpf/common/bpf_bpfel_arm64.go index 85de02229..56ea54661 100644 --- a/pkg/internal/ebpf/common/bpf_bpfel_arm64.go +++ b/pkg/internal/ebpf/common/bpf_bpfel_arm64.go @@ -90,7 +90,16 @@ type bpfHttpRequestTrace struct { _ [2]byte Conn bpfConnectionInfoT ContentLength int64 - Tp struct { + Error struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 + } + Tp struct { TraceId [16]uint8 SpanId [8]uint8 ParentId [8]uint8 diff --git a/pkg/internal/ebpf/common/bpf_bpfel_arm64.o b/pkg/internal/ebpf/common/bpf_bpfel_arm64.o index 5d3393e13..6153a04fa 100644 Binary files a/pkg/internal/ebpf/common/bpf_bpfel_arm64.o and b/pkg/internal/ebpf/common/bpf_bpfel_arm64.o differ diff --git a/pkg/internal/ebpf/common/bpf_bpfel_x86.go b/pkg/internal/ebpf/common/bpf_bpfel_x86.go index 79fb03529..fdd1dc16c 100644 --- a/pkg/internal/ebpf/common/bpf_bpfel_x86.go +++ b/pkg/internal/ebpf/common/bpf_bpfel_x86.go @@ -90,7 +90,16 @@ type bpfHttpRequestTrace struct { _ [2]byte Conn bpfConnectionInfoT ContentLength int64 - Tp struct { + Error struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 + } + Tp struct { TraceId [16]uint8 SpanId [8]uint8 ParentId [8]uint8 diff --git a/pkg/internal/ebpf/common/bpf_bpfel_x86.o b/pkg/internal/ebpf/common/bpf_bpfel_x86.o index 5d3393e13..6153a04fa 100644 Binary files a/pkg/internal/ebpf/common/bpf_bpfel_x86.o and b/pkg/internal/ebpf/common/bpf_bpfel_x86.o differ diff --git a/pkg/internal/ebpf/common/common.go b/pkg/internal/ebpf/common/common.go index 3bedc6abd..f6d4464ca 100644 --- a/pkg/internal/ebpf/common/common.go +++ b/pkg/internal/ebpf/common/common.go @@ -3,6 +3,7 @@ package ebpfcommon import ( "bufio" "bytes" + "debug/gosym" "encoding/binary" "io" "log/slog" @@ -103,7 +104,7 @@ var MisclassifiedEvents = make(chan MisclassifiedEvent) func ptlog() *slog.Logger { return slog.With("component", "ebpf.ProcessTracer") } -func ReadBPFTraceAsSpan(record *ringbuf.Record, filter ServiceFilter) (request.Span, bool, error) { +func ReadBPFTraceAsSpan(record *ringbuf.Record, filter ServiceFilter, symTab *gosym.Table) (request.Span, bool, error) { var eventType uint8 // we read the type first, depending on the type we decide what kind of record we have @@ -136,7 +137,7 @@ func ReadBPFTraceAsSpan(record *ringbuf.Record, filter ServiceFilter) (request.S return request.Span{}, true, err } - return HTTPRequestTraceToSpan(&event), false, nil + return HTTPRequestTraceToSpan(&event, symTab), false, nil } func ReadSQLRequestTraceAsSpan(record *ringbuf.Record) (request.Span, bool, error) { diff --git a/pkg/internal/ebpf/common/ringbuf.go b/pkg/internal/ebpf/common/ringbuf.go index 28c8cee7b..7243539d3 100644 --- a/pkg/internal/ebpf/common/ringbuf.go +++ b/pkg/internal/ebpf/common/ringbuf.go @@ -2,6 +2,7 @@ package ebpfcommon import ( "context" + "debug/gosym" "errors" "io" "log/slog" @@ -37,11 +38,12 @@ type ringBufForwarder struct { spansLen int access sync.Mutex ticker *time.Ticker - reader func(*ringbuf.Record, ServiceFilter) (request.Span, bool, error) + reader func(*ringbuf.Record, ServiceFilter, *gosym.Table) (request.Span, bool, error) // filter the input spans, eliminating these from processes whose PID // belong to a process that does not match the discovery policies filter ServiceFilter metrics imetrics.Reporter + symTab *gosym.Table } var singleRbf *ringBufForwarder @@ -55,6 +57,7 @@ func SharedRingbuf( filter ServiceFilter, ringbuffer *ebpf.Map, metrics imetrics.Reporter, + symTab *gosym.Table, ) func(context.Context, []io.Closer, chan<- []request.Span) { singleRbfLock.Lock() defer singleRbfLock.Unlock() @@ -67,7 +70,7 @@ func SharedRingbuf( rbf := ringBufForwarder{ cfg: cfg, logger: log, ringbuffer: ringbuffer, closers: nil, reader: ReadBPFTraceAsSpan, - filter: filter, metrics: metrics, + filter: filter, metrics: metrics, symTab: symTab, } singleRbf = &rbf return singleRbf.sharedReadAndForward @@ -77,7 +80,7 @@ func ForwardRingbuf( cfg *TracerConfig, ringbuffer *ebpf.Map, filter ServiceFilter, - reader func(*ringbuf.Record, ServiceFilter) (request.Span, bool, error), + reader func(*ringbuf.Record, ServiceFilter, *gosym.Table) (request.Span, bool, error), logger *slog.Logger, metrics imetrics.Reporter, closers ...io.Closer, @@ -170,7 +173,7 @@ func (rbf *ringBufForwarder) alreadyForwarded(ctx context.Context, _ []io.Closer func (rbf *ringBufForwarder) processAndForward(record ringbuf.Record, spansChan chan<- []request.Span) { rbf.access.Lock() defer rbf.access.Unlock() - s, ignore, err := rbf.reader(&record, rbf.filter) + s, ignore, err := rbf.reader(&record, rbf.filter, rbf.symTab) if err != nil { rbf.logger.Error("error parsing perf event", err) return diff --git a/pkg/internal/ebpf/common/spanner.go b/pkg/internal/ebpf/common/spanner.go index 0b4f5f778..adb374913 100644 --- a/pkg/internal/ebpf/common/spanner.go +++ b/pkg/internal/ebpf/common/spanner.go @@ -2,8 +2,11 @@ package ebpfcommon import ( "bytes" + "debug/gosym" + "fmt" "log/slog" "net" + "strings" trace2 "go.opentelemetry.io/otel/trace" @@ -13,7 +16,7 @@ import ( var log = slog.With("component", "goexec.spanner") -func HTTPRequestTraceToSpan(trace *HTTPRequestTrace) request.Span { +func HTTPRequestTraceToSpan(trace *HTTPRequestTrace, symTab *gosym.Table) request.Span { // From C, assuming 0-ended strings methodLen := bytes.IndexByte(trace.Method[:], 0) if methodLen < 0 { @@ -24,6 +27,10 @@ func HTTPRequestTraceToSpan(trace *HTTPRequestTrace) request.Span { if pathLen < 0 { pathLen = len(trace.Path) } + errMsgLen := bytes.IndexByte(trace.Error.ErrMsg[:], 0) + if errMsgLen < 0 { + errMsgLen = len(trace.Error.ErrMsg) + } path := string(trace.Path[:pathLen]) peer := "" @@ -57,6 +64,8 @@ func HTTPRequestTraceToSpan(trace *HTTPRequestTrace) request.Span { UserPID: trace.Pid.UserPid, Namespace: trace.Pid.Ns, }, + ErrorMessage: string(trace.Error.ErrMsg[:errMsgLen]), + ErrorStacktrace: extractErrorStacktrace(trace, symTab), } } @@ -69,6 +78,22 @@ func (trace *HTTPRequestTrace) hostInfo() (source, target string) { return src.String(), dst.String() } +func extractErrorStacktrace(trace *HTTPRequestTrace, symTab *gosym.Table) string { + var stacktrace strings.Builder + if trace.Error.UstackSz > 0 { + for _, pc := range trace.Error.Ustack { + f := symTab.PCToFunc(pc) + if f == nil { + break + } + file, line, _ := symTab.PCToLine(pc) + stacktrace.WriteString(fmt.Sprintf("%s\n", f.Name)) + stacktrace.WriteString(fmt.Sprintf("\t%s:%d\n", file, line)) + } + } + return stacktrace.String() +} + func SQLRequestTraceToSpan(trace *SQLRequestTrace) request.Span { if request.EventType(trace.Type) != request.EventTypeSQLClient { log.Warn("unknown trace type", "type", trace.Type) diff --git a/pkg/internal/ebpf/common/spanner_test.go b/pkg/internal/ebpf/common/spanner_test.go index 4a685d795..738ee3e6c 100644 --- a/pkg/internal/ebpf/common/spanner_test.go +++ b/pkg/internal/ebpf/common/spanner_test.go @@ -1,6 +1,7 @@ package ebpfcommon import ( + "debug/gosym" "testing" "github.com/stretchr/testify/assert" @@ -53,38 +54,42 @@ func assertMatches(t *testing.T, span *request.Span, method, path string, status } func TestRequestTraceParsing(t *testing.T) { + lineTable := gosym.NewLineTable([]byte{}, 0) + symTab, _ := gosym.NewTable(nil, lineTable) t.Run("Test basic parsing", func(t *testing.T) { tr := makeHTTPRequestTrace("POST", "/users", 200, 5) - s := HTTPRequestTraceToSpan(&tr) + s := HTTPRequestTraceToSpan(&tr, symTab) assertMatches(t, &s, "POST", "/users", 200, 5) }) t.Run("Test with empty path and missing peer host", func(t *testing.T) { tr := makeHTTPRequestTrace("GET", "", 403, 6) - s := HTTPRequestTraceToSpan(&tr) + s := HTTPRequestTraceToSpan(&tr, symTab) assertMatches(t, &s, "GET", "", 403, 6) }) t.Run("Test with missing peer port", func(t *testing.T) { tr := makeHTTPRequestTrace("GET", "/posts/1/1", 500, 1) - s := HTTPRequestTraceToSpan(&tr) + s := HTTPRequestTraceToSpan(&tr, symTab) assertMatches(t, &s, "GET", "/posts/1/1", 500, 1) }) t.Run("Test with invalid peer port", func(t *testing.T) { tr := makeHTTPRequestTrace("GET", "/posts/1/1", 500, 1) - s := HTTPRequestTraceToSpan(&tr) + s := HTTPRequestTraceToSpan(&tr, symTab) assertMatches(t, &s, "GET", "/posts/1/1", 500, 1) }) t.Run("Test with GRPC request", func(t *testing.T) { tr := makeGRPCRequestTrace("/posts/1/1", 2, 1) - s := HTTPRequestTraceToSpan(&tr) + s := HTTPRequestTraceToSpan(&tr, symTab) assertMatches(t, &s, "", "/posts/1/1", 2, 1) }) } func makeSpanWithTimings(goStart, start, end uint64) request.Span { + lineTable := gosym.NewLineTable([]byte{}, 0) + symTab, _ := gosym.NewTable(nil, lineTable) tr := HTTPRequestTrace{ Type: 1, Path: [100]uint8{}, @@ -94,7 +99,7 @@ func makeSpanWithTimings(goStart, start, end uint64) request.Span { EndMonotimeNs: end, } - return HTTPRequestTraceToSpan(&tr) + return HTTPRequestTraceToSpan(&tr, symTab) } func TestSpanNesting(t *testing.T) { diff --git a/pkg/internal/ebpf/goredis/goredis.go b/pkg/internal/ebpf/goredis/goredis.go index a7f17c0a3..5daeb893f 100644 --- a/pkg/internal/ebpf/goredis/goredis.go +++ b/pkg/internal/ebpf/goredis/goredis.go @@ -150,5 +150,6 @@ func (p *Tracer) Run(ctx context.Context, eventsChan chan<- []request.Span) { p.pidsFilter, p.bpfObjects.Events, p.metrics, + nil, )(ctx, append(p.closers, &p.bpfObjects), eventsChan) } diff --git a/pkg/internal/ebpf/grpc/bpf_bpfel_arm64.o b/pkg/internal/ebpf/grpc/bpf_bpfel_arm64.o index b0088b9e0..89bf1cd99 100644 Binary files a/pkg/internal/ebpf/grpc/bpf_bpfel_arm64.o and b/pkg/internal/ebpf/grpc/bpf_bpfel_arm64.o differ diff --git a/pkg/internal/ebpf/grpc/bpf_bpfel_x86.o b/pkg/internal/ebpf/grpc/bpf_bpfel_x86.o index 0bce887cf..1d07e940c 100644 Binary files a/pkg/internal/ebpf/grpc/bpf_bpfel_x86.o and b/pkg/internal/ebpf/grpc/bpf_bpfel_x86.o differ diff --git a/pkg/internal/ebpf/grpc/bpf_debug_bpfel_arm64.o b/pkg/internal/ebpf/grpc/bpf_debug_bpfel_arm64.o index afa82931c..a5aeebb92 100644 Binary files a/pkg/internal/ebpf/grpc/bpf_debug_bpfel_arm64.o and b/pkg/internal/ebpf/grpc/bpf_debug_bpfel_arm64.o differ diff --git a/pkg/internal/ebpf/grpc/bpf_debug_bpfel_x86.o b/pkg/internal/ebpf/grpc/bpf_debug_bpfel_x86.o index 3d51486aa..fd94d30fa 100644 Binary files a/pkg/internal/ebpf/grpc/bpf_debug_bpfel_x86.o and b/pkg/internal/ebpf/grpc/bpf_debug_bpfel_x86.o differ diff --git a/pkg/internal/ebpf/grpc/bpf_tp_bpfel_arm64.o b/pkg/internal/ebpf/grpc/bpf_tp_bpfel_arm64.o index 7e8f1f714..82ec2b4ff 100644 Binary files a/pkg/internal/ebpf/grpc/bpf_tp_bpfel_arm64.o and b/pkg/internal/ebpf/grpc/bpf_tp_bpfel_arm64.o differ diff --git a/pkg/internal/ebpf/grpc/bpf_tp_bpfel_x86.o b/pkg/internal/ebpf/grpc/bpf_tp_bpfel_x86.o index 89d1222f2..85716d03b 100644 Binary files a/pkg/internal/ebpf/grpc/bpf_tp_bpfel_x86.o and b/pkg/internal/ebpf/grpc/bpf_tp_bpfel_x86.o differ diff --git a/pkg/internal/ebpf/grpc/bpf_tp_debug_bpfel_arm64.o b/pkg/internal/ebpf/grpc/bpf_tp_debug_bpfel_arm64.o index 3ceeafbb7..24ea6edab 100644 Binary files a/pkg/internal/ebpf/grpc/bpf_tp_debug_bpfel_arm64.o and b/pkg/internal/ebpf/grpc/bpf_tp_debug_bpfel_arm64.o differ diff --git a/pkg/internal/ebpf/grpc/bpf_tp_debug_bpfel_x86.o b/pkg/internal/ebpf/grpc/bpf_tp_debug_bpfel_x86.o index 510180665..5b539d44d 100644 Binary files a/pkg/internal/ebpf/grpc/bpf_tp_debug_bpfel_x86.o and b/pkg/internal/ebpf/grpc/bpf_tp_debug_bpfel_x86.o differ diff --git a/pkg/internal/ebpf/grpc/grpc.go b/pkg/internal/ebpf/grpc/grpc.go index a67f07e33..e9474327c 100644 --- a/pkg/internal/ebpf/grpc/grpc.go +++ b/pkg/internal/ebpf/grpc/grpc.go @@ -199,5 +199,6 @@ func (p *Tracer) Run(ctx context.Context, eventsChan chan<- []request.Span) { p.pidsFilter, p.bpfObjects.Events, p.metrics, + nil, )(ctx, append(p.closers, &p.bpfObjects), eventsChan) } diff --git a/pkg/internal/ebpf/httpfltr/httpfltr.go b/pkg/internal/ebpf/httpfltr/httpfltr.go index 39828196b..558e2de16 100644 --- a/pkg/internal/ebpf/httpfltr/httpfltr.go +++ b/pkg/internal/ebpf/httpfltr/httpfltr.go @@ -282,6 +282,7 @@ func (p *Tracer) Run(ctx context.Context, eventsChan chan<- []request.Span) { p.pidsFilter, p.bpfObjects.Events, p.metrics, + nil, )(ctx, append(p.closers, &p.bpfObjects), eventsChan) } diff --git a/pkg/internal/ebpf/httpssl/httpssl.go b/pkg/internal/ebpf/httpssl/httpssl.go index e644a5b9c..1e603c106 100644 --- a/pkg/internal/ebpf/httpssl/httpssl.go +++ b/pkg/internal/ebpf/httpssl/httpssl.go @@ -203,5 +203,6 @@ func (p *Tracer) Run(ctx context.Context, eventsChan chan<- []request.Span) { p.pidsFilter, p.bpfObjects.Events, p.metrics, + nil, )(ctx, append(p.closers, &p.bpfObjects), eventsChan) } diff --git a/pkg/internal/ebpf/kafkago/kafkago.go b/pkg/internal/ebpf/kafkago/kafkago.go index a484ffa99..ac977c4c5 100644 --- a/pkg/internal/ebpf/kafkago/kafkago.go +++ b/pkg/internal/ebpf/kafkago/kafkago.go @@ -158,5 +158,6 @@ func (p *Tracer) Run(ctx context.Context, eventsChan chan<- []request.Span) { p.pidsFilter, p.bpfObjects.Events, p.metrics, + nil, )(ctx, append(p.closers, &p.bpfObjects), eventsChan) } diff --git a/pkg/internal/ebpf/logger/logger.go b/pkg/internal/ebpf/logger/logger.go index 34ae48f81..d988c0ff3 100644 --- a/pkg/internal/ebpf/logger/logger.go +++ b/pkg/internal/ebpf/logger/logger.go @@ -3,6 +3,7 @@ package logger import ( "bytes" "context" + "debug/gosym" "encoding/binary" "errors" "io" @@ -76,7 +77,7 @@ func (p *BPFLogger) Run(ctx context.Context) { )(ctx, nil) } -func (p *BPFLogger) processLogEvent(record *ringbuf.Record, _ ebpfcommon.ServiceFilter) (request.Span, bool, error) { +func (p *BPFLogger) processLogEvent(record *ringbuf.Record, _ ebpfcommon.ServiceFilter, _ *gosym.Table) (request.Span, bool, error) { var event BPFLogInfo err := binary.Read(bytes.NewBuffer(record.RawSample), binary.LittleEndian, &event) diff --git a/pkg/internal/ebpf/nethttp/bpf_bpfel_arm64.go b/pkg/internal/ebpf/nethttp/bpf_bpfel_arm64.go index 93f9d6317..c8fcad4c4 100644 --- a/pkg/internal/ebpf/nethttp/bpf_bpfel_arm64.go +++ b/pkg/internal/ebpf/nethttp/bpf_bpfel_arm64.go @@ -19,6 +19,16 @@ type bpfConnectionInfoT struct { D_port uint16 } +type bpfErrorEvent struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 +} + type bpfGoroutineMetadata struct { Parent uint64 Timestamp uint64 @@ -122,6 +132,7 @@ type bpfProgramSpecs struct { UprobeServeHTTPReturns *ebpf.ProgramSpec `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.ProgramSpec `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.ProgramSpec `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.ProgramSpec `ebpf:"uprobe_error"` UprobeExecDC *ebpf.ProgramSpec `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -144,6 +155,7 @@ type bpfMapSpecs struct { Events *ebpf.MapSpec `ebpf:"events"` GoTraceMap *ebpf.MapSpec `ebpf:"go_trace_map"` GolangMapbucketStorageMap *ebpf.MapSpec `ebpf:"golang_mapbucket_storage_map"` + LastError *ebpf.MapSpec `ebpf:"last_error"` OngoingClientConnections *ebpf.MapSpec `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.MapSpec `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.MapSpec `ebpf:"ongoing_http_client_requests"` @@ -176,6 +188,7 @@ type bpfMaps struct { Events *ebpf.Map `ebpf:"events"` GoTraceMap *ebpf.Map `ebpf:"go_trace_map"` GolangMapbucketStorageMap *ebpf.Map `ebpf:"golang_mapbucket_storage_map"` + LastError *ebpf.Map `ebpf:"last_error"` OngoingClientConnections *ebpf.Map `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.Map `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.Map `ebpf:"ongoing_http_client_requests"` @@ -191,6 +204,7 @@ func (m *bpfMaps) Close() error { m.Events, m.GoTraceMap, m.GolangMapbucketStorageMap, + m.LastError, m.OngoingClientConnections, m.OngoingGoroutines, m.OngoingHttpClientRequests, @@ -210,6 +224,7 @@ type bpfPrograms struct { UprobeServeHTTPReturns *ebpf.Program `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.Program `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.Program `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.Program `ebpf:"uprobe_error"` UprobeExecDC *ebpf.Program `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -231,6 +246,7 @@ func (p *bpfPrograms) Close() error { p.UprobeServeHTTPReturns, p.UprobeConnServe, p.UprobeConnServeRet, + p.UprobeError, p.UprobeExecDC, p.UprobeHttp2FramerWriteHeaders, p.UprobeHttp2FramerWriteHeadersReturns, diff --git a/pkg/internal/ebpf/nethttp/bpf_bpfel_arm64.o b/pkg/internal/ebpf/nethttp/bpf_bpfel_arm64.o index fed524eee..b02efd4f9 100644 Binary files a/pkg/internal/ebpf/nethttp/bpf_bpfel_arm64.o and b/pkg/internal/ebpf/nethttp/bpf_bpfel_arm64.o differ diff --git a/pkg/internal/ebpf/nethttp/bpf_bpfel_x86.go b/pkg/internal/ebpf/nethttp/bpf_bpfel_x86.go index f0bb71eaf..6a734a1f4 100644 --- a/pkg/internal/ebpf/nethttp/bpf_bpfel_x86.go +++ b/pkg/internal/ebpf/nethttp/bpf_bpfel_x86.go @@ -19,6 +19,16 @@ type bpfConnectionInfoT struct { D_port uint16 } +type bpfErrorEvent struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 +} + type bpfGoroutineMetadata struct { Parent uint64 Timestamp uint64 @@ -122,6 +132,7 @@ type bpfProgramSpecs struct { UprobeServeHTTPReturns *ebpf.ProgramSpec `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.ProgramSpec `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.ProgramSpec `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.ProgramSpec `ebpf:"uprobe_error"` UprobeExecDC *ebpf.ProgramSpec `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -144,6 +155,7 @@ type bpfMapSpecs struct { Events *ebpf.MapSpec `ebpf:"events"` GoTraceMap *ebpf.MapSpec `ebpf:"go_trace_map"` GolangMapbucketStorageMap *ebpf.MapSpec `ebpf:"golang_mapbucket_storage_map"` + LastError *ebpf.MapSpec `ebpf:"last_error"` OngoingClientConnections *ebpf.MapSpec `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.MapSpec `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.MapSpec `ebpf:"ongoing_http_client_requests"` @@ -176,6 +188,7 @@ type bpfMaps struct { Events *ebpf.Map `ebpf:"events"` GoTraceMap *ebpf.Map `ebpf:"go_trace_map"` GolangMapbucketStorageMap *ebpf.Map `ebpf:"golang_mapbucket_storage_map"` + LastError *ebpf.Map `ebpf:"last_error"` OngoingClientConnections *ebpf.Map `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.Map `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.Map `ebpf:"ongoing_http_client_requests"` @@ -191,6 +204,7 @@ func (m *bpfMaps) Close() error { m.Events, m.GoTraceMap, m.GolangMapbucketStorageMap, + m.LastError, m.OngoingClientConnections, m.OngoingGoroutines, m.OngoingHttpClientRequests, @@ -210,6 +224,7 @@ type bpfPrograms struct { UprobeServeHTTPReturns *ebpf.Program `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.Program `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.Program `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.Program `ebpf:"uprobe_error"` UprobeExecDC *ebpf.Program `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -231,6 +246,7 @@ func (p *bpfPrograms) Close() error { p.UprobeServeHTTPReturns, p.UprobeConnServe, p.UprobeConnServeRet, + p.UprobeError, p.UprobeExecDC, p.UprobeHttp2FramerWriteHeaders, p.UprobeHttp2FramerWriteHeadersReturns, diff --git a/pkg/internal/ebpf/nethttp/bpf_bpfel_x86.o b/pkg/internal/ebpf/nethttp/bpf_bpfel_x86.o index 6c38674ef..955d7b6c3 100644 Binary files a/pkg/internal/ebpf/nethttp/bpf_bpfel_x86.o and b/pkg/internal/ebpf/nethttp/bpf_bpfel_x86.o differ diff --git a/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_arm64.go b/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_arm64.go index 726ab6b0d..0bd3153e9 100644 --- a/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_arm64.go +++ b/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_arm64.go @@ -19,6 +19,16 @@ type bpf_debugConnectionInfoT struct { D_port uint16 } +type bpf_debugErrorEvent struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 +} + type bpf_debugGoroutineMetadata struct { Parent uint64 Timestamp uint64 @@ -122,6 +132,7 @@ type bpf_debugProgramSpecs struct { UprobeServeHTTPReturns *ebpf.ProgramSpec `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.ProgramSpec `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.ProgramSpec `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.ProgramSpec `ebpf:"uprobe_error"` UprobeExecDC *ebpf.ProgramSpec `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -145,6 +156,7 @@ type bpf_debugMapSpecs struct { Events *ebpf.MapSpec `ebpf:"events"` GoTraceMap *ebpf.MapSpec `ebpf:"go_trace_map"` GolangMapbucketStorageMap *ebpf.MapSpec `ebpf:"golang_mapbucket_storage_map"` + LastError *ebpf.MapSpec `ebpf:"last_error"` OngoingClientConnections *ebpf.MapSpec `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.MapSpec `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.MapSpec `ebpf:"ongoing_http_client_requests"` @@ -178,6 +190,7 @@ type bpf_debugMaps struct { Events *ebpf.Map `ebpf:"events"` GoTraceMap *ebpf.Map `ebpf:"go_trace_map"` GolangMapbucketStorageMap *ebpf.Map `ebpf:"golang_mapbucket_storage_map"` + LastError *ebpf.Map `ebpf:"last_error"` OngoingClientConnections *ebpf.Map `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.Map `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.Map `ebpf:"ongoing_http_client_requests"` @@ -194,6 +207,7 @@ func (m *bpf_debugMaps) Close() error { m.Events, m.GoTraceMap, m.GolangMapbucketStorageMap, + m.LastError, m.OngoingClientConnections, m.OngoingGoroutines, m.OngoingHttpClientRequests, @@ -213,6 +227,7 @@ type bpf_debugPrograms struct { UprobeServeHTTPReturns *ebpf.Program `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.Program `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.Program `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.Program `ebpf:"uprobe_error"` UprobeExecDC *ebpf.Program `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -234,6 +249,7 @@ func (p *bpf_debugPrograms) Close() error { p.UprobeServeHTTPReturns, p.UprobeConnServe, p.UprobeConnServeRet, + p.UprobeError, p.UprobeExecDC, p.UprobeHttp2FramerWriteHeaders, p.UprobeHttp2FramerWriteHeadersReturns, diff --git a/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_arm64.o b/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_arm64.o index 97e99caf9..a536f16c0 100644 Binary files a/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_arm64.o and b/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_arm64.o differ diff --git a/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_x86.go b/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_x86.go index a9d759052..3e806ff69 100644 --- a/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_x86.go +++ b/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_x86.go @@ -19,6 +19,16 @@ type bpf_debugConnectionInfoT struct { D_port uint16 } +type bpf_debugErrorEvent struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 +} + type bpf_debugGoroutineMetadata struct { Parent uint64 Timestamp uint64 @@ -122,6 +132,7 @@ type bpf_debugProgramSpecs struct { UprobeServeHTTPReturns *ebpf.ProgramSpec `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.ProgramSpec `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.ProgramSpec `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.ProgramSpec `ebpf:"uprobe_error"` UprobeExecDC *ebpf.ProgramSpec `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -145,6 +156,7 @@ type bpf_debugMapSpecs struct { Events *ebpf.MapSpec `ebpf:"events"` GoTraceMap *ebpf.MapSpec `ebpf:"go_trace_map"` GolangMapbucketStorageMap *ebpf.MapSpec `ebpf:"golang_mapbucket_storage_map"` + LastError *ebpf.MapSpec `ebpf:"last_error"` OngoingClientConnections *ebpf.MapSpec `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.MapSpec `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.MapSpec `ebpf:"ongoing_http_client_requests"` @@ -178,6 +190,7 @@ type bpf_debugMaps struct { Events *ebpf.Map `ebpf:"events"` GoTraceMap *ebpf.Map `ebpf:"go_trace_map"` GolangMapbucketStorageMap *ebpf.Map `ebpf:"golang_mapbucket_storage_map"` + LastError *ebpf.Map `ebpf:"last_error"` OngoingClientConnections *ebpf.Map `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.Map `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.Map `ebpf:"ongoing_http_client_requests"` @@ -194,6 +207,7 @@ func (m *bpf_debugMaps) Close() error { m.Events, m.GoTraceMap, m.GolangMapbucketStorageMap, + m.LastError, m.OngoingClientConnections, m.OngoingGoroutines, m.OngoingHttpClientRequests, @@ -213,6 +227,7 @@ type bpf_debugPrograms struct { UprobeServeHTTPReturns *ebpf.Program `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.Program `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.Program `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.Program `ebpf:"uprobe_error"` UprobeExecDC *ebpf.Program `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -234,6 +249,7 @@ func (p *bpf_debugPrograms) Close() error { p.UprobeServeHTTPReturns, p.UprobeConnServe, p.UprobeConnServeRet, + p.UprobeError, p.UprobeExecDC, p.UprobeHttp2FramerWriteHeaders, p.UprobeHttp2FramerWriteHeadersReturns, diff --git a/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_x86.o b/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_x86.o index 19a9fb674..fe4da627f 100644 Binary files a/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_x86.o and b/pkg/internal/ebpf/nethttp/bpf_debug_bpfel_x86.o differ diff --git a/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_arm64.go b/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_arm64.go index a92c639b8..10b963b66 100644 --- a/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_arm64.go +++ b/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_arm64.go @@ -19,6 +19,16 @@ type bpf_tpConnectionInfoT struct { D_port uint16 } +type bpf_tpErrorEvent struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 +} + type bpf_tpFramerFuncInvocationT struct { FramerPtr uint64 Tp bpf_tpTpInfoT @@ -128,6 +138,7 @@ type bpf_tpProgramSpecs struct { UprobeServeHTTPReturns *ebpf.ProgramSpec `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.ProgramSpec `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.ProgramSpec `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.ProgramSpec `ebpf:"uprobe_error"` UprobeExecDC *ebpf.ProgramSpec `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -153,6 +164,7 @@ type bpf_tpMapSpecs struct { GolangMapbucketStorageMap *ebpf.MapSpec `ebpf:"golang_mapbucket_storage_map"` HeaderReqMap *ebpf.MapSpec `ebpf:"header_req_map"` Http2ReqMap *ebpf.MapSpec `ebpf:"http2_req_map"` + LastError *ebpf.MapSpec `ebpf:"last_error"` OngoingClientConnections *ebpf.MapSpec `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.MapSpec `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.MapSpec `ebpf:"ongoing_http_client_requests"` @@ -188,6 +200,7 @@ type bpf_tpMaps struct { GolangMapbucketStorageMap *ebpf.Map `ebpf:"golang_mapbucket_storage_map"` HeaderReqMap *ebpf.Map `ebpf:"header_req_map"` Http2ReqMap *ebpf.Map `ebpf:"http2_req_map"` + LastError *ebpf.Map `ebpf:"last_error"` OngoingClientConnections *ebpf.Map `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.Map `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.Map `ebpf:"ongoing_http_client_requests"` @@ -206,6 +219,7 @@ func (m *bpf_tpMaps) Close() error { m.GolangMapbucketStorageMap, m.HeaderReqMap, m.Http2ReqMap, + m.LastError, m.OngoingClientConnections, m.OngoingGoroutines, m.OngoingHttpClientRequests, @@ -225,6 +239,7 @@ type bpf_tpPrograms struct { UprobeServeHTTPReturns *ebpf.Program `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.Program `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.Program `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.Program `ebpf:"uprobe_error"` UprobeExecDC *ebpf.Program `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -246,6 +261,7 @@ func (p *bpf_tpPrograms) Close() error { p.UprobeServeHTTPReturns, p.UprobeConnServe, p.UprobeConnServeRet, + p.UprobeError, p.UprobeExecDC, p.UprobeHttp2FramerWriteHeaders, p.UprobeHttp2FramerWriteHeadersReturns, diff --git a/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_arm64.o b/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_arm64.o index 87ff6e8bc..3d596f181 100644 Binary files a/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_arm64.o and b/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_arm64.o differ diff --git a/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_x86.go b/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_x86.go index a8d396949..55af85198 100644 --- a/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_x86.go +++ b/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_x86.go @@ -19,6 +19,16 @@ type bpf_tpConnectionInfoT struct { D_port uint16 } +type bpf_tpErrorEvent struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 +} + type bpf_tpFramerFuncInvocationT struct { FramerPtr uint64 Tp bpf_tpTpInfoT @@ -128,6 +138,7 @@ type bpf_tpProgramSpecs struct { UprobeServeHTTPReturns *ebpf.ProgramSpec `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.ProgramSpec `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.ProgramSpec `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.ProgramSpec `ebpf:"uprobe_error"` UprobeExecDC *ebpf.ProgramSpec `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -153,6 +164,7 @@ type bpf_tpMapSpecs struct { GolangMapbucketStorageMap *ebpf.MapSpec `ebpf:"golang_mapbucket_storage_map"` HeaderReqMap *ebpf.MapSpec `ebpf:"header_req_map"` Http2ReqMap *ebpf.MapSpec `ebpf:"http2_req_map"` + LastError *ebpf.MapSpec `ebpf:"last_error"` OngoingClientConnections *ebpf.MapSpec `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.MapSpec `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.MapSpec `ebpf:"ongoing_http_client_requests"` @@ -188,6 +200,7 @@ type bpf_tpMaps struct { GolangMapbucketStorageMap *ebpf.Map `ebpf:"golang_mapbucket_storage_map"` HeaderReqMap *ebpf.Map `ebpf:"header_req_map"` Http2ReqMap *ebpf.Map `ebpf:"http2_req_map"` + LastError *ebpf.Map `ebpf:"last_error"` OngoingClientConnections *ebpf.Map `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.Map `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.Map `ebpf:"ongoing_http_client_requests"` @@ -206,6 +219,7 @@ func (m *bpf_tpMaps) Close() error { m.GolangMapbucketStorageMap, m.HeaderReqMap, m.Http2ReqMap, + m.LastError, m.OngoingClientConnections, m.OngoingGoroutines, m.OngoingHttpClientRequests, @@ -225,6 +239,7 @@ type bpf_tpPrograms struct { UprobeServeHTTPReturns *ebpf.Program `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.Program `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.Program `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.Program `ebpf:"uprobe_error"` UprobeExecDC *ebpf.Program `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -246,6 +261,7 @@ func (p *bpf_tpPrograms) Close() error { p.UprobeServeHTTPReturns, p.UprobeConnServe, p.UprobeConnServeRet, + p.UprobeError, p.UprobeExecDC, p.UprobeHttp2FramerWriteHeaders, p.UprobeHttp2FramerWriteHeadersReturns, diff --git a/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_x86.o b/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_x86.o index 22b31db40..d1b9e29f5 100644 Binary files a/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_x86.o and b/pkg/internal/ebpf/nethttp/bpf_tp_bpfel_x86.o differ diff --git a/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_arm64.go b/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_arm64.go index f3b070c05..fdd4c0f73 100644 --- a/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_arm64.go +++ b/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_arm64.go @@ -19,6 +19,16 @@ type bpf_tp_debugConnectionInfoT struct { D_port uint16 } +type bpf_tp_debugErrorEvent struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 +} + type bpf_tp_debugFramerFuncInvocationT struct { FramerPtr uint64 Tp bpf_tp_debugTpInfoT @@ -128,6 +138,7 @@ type bpf_tp_debugProgramSpecs struct { UprobeServeHTTPReturns *ebpf.ProgramSpec `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.ProgramSpec `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.ProgramSpec `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.ProgramSpec `ebpf:"uprobe_error"` UprobeExecDC *ebpf.ProgramSpec `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -154,6 +165,7 @@ type bpf_tp_debugMapSpecs struct { GolangMapbucketStorageMap *ebpf.MapSpec `ebpf:"golang_mapbucket_storage_map"` HeaderReqMap *ebpf.MapSpec `ebpf:"header_req_map"` Http2ReqMap *ebpf.MapSpec `ebpf:"http2_req_map"` + LastError *ebpf.MapSpec `ebpf:"last_error"` OngoingClientConnections *ebpf.MapSpec `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.MapSpec `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.MapSpec `ebpf:"ongoing_http_client_requests"` @@ -190,6 +202,7 @@ type bpf_tp_debugMaps struct { GolangMapbucketStorageMap *ebpf.Map `ebpf:"golang_mapbucket_storage_map"` HeaderReqMap *ebpf.Map `ebpf:"header_req_map"` Http2ReqMap *ebpf.Map `ebpf:"http2_req_map"` + LastError *ebpf.Map `ebpf:"last_error"` OngoingClientConnections *ebpf.Map `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.Map `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.Map `ebpf:"ongoing_http_client_requests"` @@ -209,6 +222,7 @@ func (m *bpf_tp_debugMaps) Close() error { m.GolangMapbucketStorageMap, m.HeaderReqMap, m.Http2ReqMap, + m.LastError, m.OngoingClientConnections, m.OngoingGoroutines, m.OngoingHttpClientRequests, @@ -228,6 +242,7 @@ type bpf_tp_debugPrograms struct { UprobeServeHTTPReturns *ebpf.Program `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.Program `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.Program `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.Program `ebpf:"uprobe_error"` UprobeExecDC *ebpf.Program `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -249,6 +264,7 @@ func (p *bpf_tp_debugPrograms) Close() error { p.UprobeServeHTTPReturns, p.UprobeConnServe, p.UprobeConnServeRet, + p.UprobeError, p.UprobeExecDC, p.UprobeHttp2FramerWriteHeaders, p.UprobeHttp2FramerWriteHeadersReturns, diff --git a/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_arm64.o b/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_arm64.o index 31eb71c8c..da7ebce3a 100644 Binary files a/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_arm64.o and b/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_arm64.o differ diff --git a/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_x86.go b/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_x86.go index 7e78c639c..2d0ae1d88 100644 --- a/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_x86.go +++ b/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_x86.go @@ -19,6 +19,16 @@ type bpf_tp_debugConnectionInfoT struct { D_port uint16 } +type bpf_tp_debugErrorEvent struct { + Pid uint32 + CpuId uint32 + Comm [16]int8 + UstackSz int32 + _ [4]byte + Ustack [32]uint64 + ErrMsg [32]uint8 +} + type bpf_tp_debugFramerFuncInvocationT struct { FramerPtr uint64 Tp bpf_tp_debugTpInfoT @@ -128,6 +138,7 @@ type bpf_tp_debugProgramSpecs struct { UprobeServeHTTPReturns *ebpf.ProgramSpec `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.ProgramSpec `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.ProgramSpec `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.ProgramSpec `ebpf:"uprobe_error"` UprobeExecDC *ebpf.ProgramSpec `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.ProgramSpec `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -154,6 +165,7 @@ type bpf_tp_debugMapSpecs struct { GolangMapbucketStorageMap *ebpf.MapSpec `ebpf:"golang_mapbucket_storage_map"` HeaderReqMap *ebpf.MapSpec `ebpf:"header_req_map"` Http2ReqMap *ebpf.MapSpec `ebpf:"http2_req_map"` + LastError *ebpf.MapSpec `ebpf:"last_error"` OngoingClientConnections *ebpf.MapSpec `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.MapSpec `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.MapSpec `ebpf:"ongoing_http_client_requests"` @@ -190,6 +202,7 @@ type bpf_tp_debugMaps struct { GolangMapbucketStorageMap *ebpf.Map `ebpf:"golang_mapbucket_storage_map"` HeaderReqMap *ebpf.Map `ebpf:"header_req_map"` Http2ReqMap *ebpf.Map `ebpf:"http2_req_map"` + LastError *ebpf.Map `ebpf:"last_error"` OngoingClientConnections *ebpf.Map `ebpf:"ongoing_client_connections"` OngoingGoroutines *ebpf.Map `ebpf:"ongoing_goroutines"` OngoingHttpClientRequests *ebpf.Map `ebpf:"ongoing_http_client_requests"` @@ -209,6 +222,7 @@ func (m *bpf_tp_debugMaps) Close() error { m.GolangMapbucketStorageMap, m.HeaderReqMap, m.Http2ReqMap, + m.LastError, m.OngoingClientConnections, m.OngoingGoroutines, m.OngoingHttpClientRequests, @@ -228,6 +242,7 @@ type bpf_tp_debugPrograms struct { UprobeServeHTTPReturns *ebpf.Program `ebpf:"uprobe_ServeHTTPReturns"` UprobeConnServe *ebpf.Program `ebpf:"uprobe_connServe"` UprobeConnServeRet *ebpf.Program `ebpf:"uprobe_connServeRet"` + UprobeError *ebpf.Program `ebpf:"uprobe_error"` UprobeExecDC *ebpf.Program `ebpf:"uprobe_execDC"` UprobeHttp2FramerWriteHeaders *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders"` UprobeHttp2FramerWriteHeadersReturns *ebpf.Program `ebpf:"uprobe_http2FramerWriteHeaders_returns"` @@ -249,6 +264,7 @@ func (p *bpf_tp_debugPrograms) Close() error { p.UprobeServeHTTPReturns, p.UprobeConnServe, p.UprobeConnServeRet, + p.UprobeError, p.UprobeExecDC, p.UprobeHttp2FramerWriteHeaders, p.UprobeHttp2FramerWriteHeadersReturns, diff --git a/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_x86.o b/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_x86.o index f9af4655c..f58fe043a 100644 Binary files a/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_x86.o and b/pkg/internal/ebpf/nethttp/bpf_tp_debug_bpfel_x86.o differ diff --git a/pkg/internal/ebpf/nethttp/nethttp.go b/pkg/internal/ebpf/nethttp/nethttp.go index ac7c624c5..87aa8c651 100644 --- a/pkg/internal/ebpf/nethttp/nethttp.go +++ b/pkg/internal/ebpf/nethttp/nethttp.go @@ -16,6 +16,7 @@ package nethttp import ( "context" + "debug/gosym" "io" "log/slog" "unsafe" @@ -43,15 +44,17 @@ type Tracer struct { metrics imetrics.Reporter bpfObjects bpfObjects closers []io.Closer + symTab *gosym.Table } -func New(cfg *beyla.Config, metrics imetrics.Reporter) *Tracer { +func New(cfg *beyla.Config, metrics imetrics.Reporter, symTab *gosym.Table) *Tracer { log := slog.With("component", "nethttp.Tracer") return &Tracer{ log: log, pidsFilter: ebpfcommon.CommonPIDsFilter(cfg.Discovery.SystemWide), cfg: &cfg.EBPF, metrics: metrics, + symTab: symTab, } } @@ -184,6 +187,9 @@ func (p *Tracer) GoProbes() map[string]ebpfcommon.FunctionPrograms { Start: p.bpfObjects.UprobeExecDC, End: p.bpfObjects.UprobeQueryReturn, }, + "fmt.Errorf": { + Start: p.bpfObjects.UprobeError, + }, } if p.supportsContextPropagation() { @@ -227,5 +233,6 @@ func (p *Tracer) Run(ctx context.Context, eventsChan chan<- []request.Span) { p.pidsFilter, p.bpfObjects.Events, p.metrics, + p.symTab, )(ctx, append(p.closers, &p.bpfObjects), eventsChan) } diff --git a/pkg/internal/ebpf/nodejs/nodejs.go b/pkg/internal/ebpf/nodejs/nodejs.go index dcaeed220..bfb258406 100644 --- a/pkg/internal/ebpf/nodejs/nodejs.go +++ b/pkg/internal/ebpf/nodejs/nodejs.go @@ -142,5 +142,6 @@ func (p *Tracer) Run(ctx context.Context, eventsChan chan<- []request.Span) { p.pidsFilter, p.bpfObjects.Events, p.metrics, + nil, )(ctx, append(p.closers, &p.bpfObjects), eventsChan) } diff --git a/pkg/internal/ebpf/sarama/sarama.go b/pkg/internal/ebpf/sarama/sarama.go index aefb23023..72107abe3 100644 --- a/pkg/internal/ebpf/sarama/sarama.go +++ b/pkg/internal/ebpf/sarama/sarama.go @@ -144,6 +144,7 @@ func (p *Tracer) Run(ctx context.Context, eventsChan chan<- []request.Span) { p.pidsFilter, p.bpfObjects.Events, p.metrics, + nil, )(ctx, append(p.closers, &p.bpfObjects), eventsChan) } @@ -175,5 +176,6 @@ func (p *ShopifyKafkaTracer) Run(ctx context.Context, eventsChan chan<- []reques p.pidsFilter, p.bpfObjects.Events, p.metrics, + nil, )(ctx, append(p.closers, &p.bpfObjects), eventsChan) } diff --git a/pkg/internal/ebpf/watcher/watcher.go b/pkg/internal/ebpf/watcher/watcher.go index 87967f20e..6f6609e55 100644 --- a/pkg/internal/ebpf/watcher/watcher.go +++ b/pkg/internal/ebpf/watcher/watcher.go @@ -3,6 +3,7 @@ package watcher import ( "bytes" "context" + "debug/gosym" "encoding/binary" "io" "log/slog" @@ -96,7 +97,7 @@ func (p *Watcher) Run(ctx context.Context) { )(ctx, nil) } -func (p *Watcher) processWatchEvent(record *ringbuf.Record, _ ebpfcommon.ServiceFilter) (request.Span, bool, error) { +func (p *Watcher) processWatchEvent(record *ringbuf.Record, _ ebpfcommon.ServiceFilter, _ *gosym.Table) (request.Span, bool, error) { var flags uint64 var event BPFWatchInfo diff --git a/pkg/internal/export/alloy/traces.go b/pkg/internal/export/alloy/traces.go index c29358fad..168db59a4 100644 --- a/pkg/internal/export/alloy/traces.go +++ b/pkg/internal/export/alloy/traces.go @@ -13,18 +13,18 @@ import ( ) // TracesReceiver creates a terminal node that consumes request.Spans and sends OpenTelemetry traces to the configured consumers. -func TracesReceiver(ctx context.Context, cfg *beyla.TracesReceiverConfig, userAttribSelection attributes.Selection) pipe.FinalProvider[[]request.Span] { +func TracesReceiver(ctx context.Context, cfg *beyla.Config, userAttribSelection attributes.Selection) pipe.FinalProvider[[]request.Span] { return (&tracesReceiver{ctx: ctx, cfg: cfg, attributes: userAttribSelection}).provideLoop } type tracesReceiver struct { ctx context.Context - cfg *beyla.TracesReceiverConfig + cfg *beyla.Config attributes attributes.Selection } func (tr *tracesReceiver) provideLoop() (pipe.FinalFunc[[]request.Span], error) { - if !tr.cfg.Enabled() { + if !tr.cfg.TracesReceiver.Enabled() { return pipe.IgnoreFinal[[]request.Span](), nil } return func(in <-chan []request.Span) { @@ -41,8 +41,8 @@ func (tr *tracesReceiver) provideLoop() (pipe.FinalFunc[[]request.Span], error) continue } - for _, tc := range tr.cfg.Traces { - traces := otel.GenerateTraces(span, traceAttrs) + for _, tc := range tr.cfg.TracesReceiver.Traces { + traces := otel.GenerateTraces(tr.cfg.Traces, span, traceAttrs) err := tc.ConsumeTraces(tr.ctx, traces) if err != nil { slog.Error("error sending trace to consumer", "error", err) diff --git a/pkg/internal/export/debug/debug.go b/pkg/internal/export/debug/debug.go index 8820e3d83..6bea83261 100644 --- a/pkg/internal/export/debug/debug.go +++ b/pkg/internal/export/debug/debug.go @@ -47,6 +47,9 @@ func printFunc() (pipe.FinalFunc[[]request.Span], error) { spans[i].ServiceID.SDKLanguage.String(), traceparent(&spans[i]), ) + if spans[i].ErrorMessage != "" { + fmt.Printf("error_message=%s stacktrace=\n%s\n", spans[i].ErrorMessage, spans[i].ErrorStacktrace) + } } } }, nil diff --git a/pkg/internal/export/otel/traces.go b/pkg/internal/export/otel/traces.go index ece50c0d9..addcf5d33 100644 --- a/pkg/internal/export/otel/traces.go +++ b/pkg/internal/export/otel/traces.go @@ -80,6 +80,9 @@ type TracesConfig struct { // BackOffMaxElapsedTime is the maximum amount of time (including retries) spent trying to send a request/batch. BackOffMaxElapsedTime time.Duration `yaml:"backoff_max_elapsed_time" env:"BEYLA_BACKOFF_MAX_ELAPSED_TIME"` + // ReportExceptionEvents enables the reporting of exception events. + ReportExceptionEvents bool `yaml:"report_exception_events" env:"BEYLA_TRACES_REPORT_EXCEPTION_EVENTS"` + ReportersCacheLen int `yaml:"reporters_cache_len" env:"BEYLA_TRACES_REPORT_CACHE_LEN"` // SDKLogLevel works independently from the global LogLevel because it prints GBs of logs in Debug mode @@ -196,7 +199,7 @@ func (tr *tracesOTELReceiver) provideLoop() (pipe.FinalFunc[[]request.Span], err if span.IgnoreSpan == request.IgnoreTraces || !tr.acceptSpan(span) { continue } - traces := GenerateTraces(span, traceAttrs) + traces := GenerateTraces(tr.cfg, span, traceAttrs) err := exp.ConsumeTraces(tr.ctx, traces) if err != nil { slog.Error("error sending trace to consumer", "error", err) @@ -363,7 +366,7 @@ func getRetrySettings(cfg TracesConfig) configretry.BackOffConfig { } // GenerateTraces creates a ptrace.Traces from a request.Span -func GenerateTraces(span *request.Span, userAttrs map[attr.Name]struct{}) ptrace.Traces { +func GenerateTraces(cfg TracesConfig, span *request.Span, userAttrs map[attr.Name]struct{}) ptrace.Traces { t := span.Timings() start := spanStartTime(t) hasSubSpans := t.Start.After(start) @@ -404,6 +407,15 @@ func GenerateTraces(span *request.Span, userAttrs map[attr.Name]struct{}) ptrace m := attrsToMap(attrs) m.CopyTo(s.Attributes()) + // Set error message and stacktrace + if cfg.ReportExceptionEvents && span.ErrorMessage != "" { + e := s.Events().AppendEmpty() + e.SetName(semconv.ExceptionEventName) + e.Attributes().PutStr(string(semconv.ExceptionMessageKey), span.ErrorMessage) + e.Attributes().PutStr(string(semconv.ExceptionTypeKey), "error") + e.Attributes().PutStr(string(semconv.ExceptionStacktraceKey), span.ErrorStacktrace) + } + // Set status code statusCode := codeToStatusCode(request.SpanStatusCode(span)) s.Status().SetCode(statusCode) diff --git a/pkg/internal/export/otel/traces_test.go b/pkg/internal/export/otel/traces_test.go index 3d6dfeb99..8a87f994b 100644 --- a/pkg/internal/export/otel/traces_test.go +++ b/pkg/internal/export/otel/traces_test.go @@ -318,18 +318,23 @@ func TestGenerateTraces(t *testing.T) { spanID, _ := trace.SpanIDFromHex("89cbc1f60aab3b01") traceID, _ := trace.TraceIDFromHex("eae56fbbec9505c102e8aabfc6b5c481") span := &request.Span{ - Type: request.EventTypeHTTP, - RequestStart: start.UnixNano(), - Start: start.Add(time.Second).UnixNano(), - End: start.Add(3 * time.Second).UnixNano(), - Method: "GET", - Route: "/test", - Status: 200, - ParentSpanID: parentSpanID, - TraceID: traceID, - SpanID: spanID, + Type: request.EventTypeHTTP, + RequestStart: start.UnixNano(), + Start: start.Add(time.Second).UnixNano(), + End: start.Add(3 * time.Second).UnixNano(), + Method: "GET", + Route: "/test", + Status: 200, + ParentSpanID: parentSpanID, + TraceID: traceID, + SpanID: spanID, + ErrorMessage: "crash", + ErrorStacktrace: "function\nline", } - traces := GenerateTraces(span, map[attr.Name]struct{}{}) + cfg := TracesConfig{ + ReportExceptionEvents: true, + } + traces := GenerateTraces(cfg, span, map[attr.Name]struct{}{}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -357,6 +362,13 @@ func TestGenerateTraces(t *testing.T) { assert.NotEqual(t, spans.At(0).SpanID().String(), spans.At(1).SpanID().String()) assert.NotEqual(t, spans.At(1).SpanID().String(), spans.At(2).SpanID().String()) + + e := spans.At(2).Events().At(0) + val, _ := e.Attributes().Get(string(semconv.ExceptionMessageKey)) + assert.Equal(t, "crash", val.AsString()) + val, _ = e.Attributes().Get(string(semconv.ExceptionStacktraceKey)) + assert.Equal(t, "function\nline", val.AsString()) + }) t.Run("test with subtraces - ids set bpf layer", func(t *testing.T) { @@ -374,7 +386,8 @@ func TestGenerateTraces(t *testing.T) { SpanID: spanID, TraceID: traceID, } - traces := GenerateTraces(span, map[attr.Name]struct{}{}) + cfg := TracesConfig{} + traces := GenerateTraces(cfg, span, map[attr.Name]struct{}{}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -410,7 +423,8 @@ func TestGenerateTraces(t *testing.T) { Route: "/test", Status: 200, } - traces := GenerateTraces(span, map[attr.Name]struct{}{}) + cfg := TracesConfig{} + traces := GenerateTraces(cfg, span, map[attr.Name]struct{}{}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -447,7 +461,8 @@ func TestGenerateTraces(t *testing.T) { SpanID: spanID, TraceID: traceID, } - traces := GenerateTraces(span, map[attr.Name]struct{}{}) + cfg := TracesConfig{} + traces := GenerateTraces(cfg, span, map[attr.Name]struct{}{}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -472,7 +487,8 @@ func TestGenerateTraces(t *testing.T) { ParentSpanID: parentSpanID, TraceID: traceID, } - traces := GenerateTraces(span, map[attr.Name]struct{}{}) + cfg := TracesConfig{} + traces := GenerateTraces(cfg, span, map[attr.Name]struct{}{}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -492,7 +508,8 @@ func TestGenerateTraces(t *testing.T) { Method: "GET", Route: "/test", } - traces := GenerateTraces(span, map[attr.Name]struct{}{}) + cfg := TracesConfig{} + traces := GenerateTraces(cfg, span, map[attr.Name]struct{}{}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -508,7 +525,8 @@ func TestGenerateTraces(t *testing.T) { func TestGenerateTracesAttributes(t *testing.T) { t.Run("test SQL trace generation, no statement", func(t *testing.T) { span := makeSQLRequestSpan("SELECT password FROM credentials WHERE username=\"bill\"") - traces := GenerateTraces(&span, map[attr.Name]struct{}{}) + cfg := TracesConfig{} + traces := GenerateTraces(cfg, &span, map[attr.Name]struct{}{}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -529,7 +547,8 @@ func TestGenerateTracesAttributes(t *testing.T) { t.Run("test SQL trace generation, unknown attribute", func(t *testing.T) { span := makeSQLRequestSpan("SELECT password FROM credentials WHERE username=\"bill\"") - traces := GenerateTraces(&span, map[attr.Name]struct{}{"db.operation.name": {}}) + cfg := TracesConfig{} + traces := GenerateTraces(cfg, &span, map[attr.Name]struct{}{"db.operation.name": {}}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -550,7 +569,8 @@ func TestGenerateTracesAttributes(t *testing.T) { t.Run("test SQL trace generation, unknown attribute", func(t *testing.T) { span := makeSQLRequestSpan("SELECT password FROM credentials WHERE username=\"bill\"") - traces := GenerateTraces(&span, map[attr.Name]struct{}{attr.DBQueryText: {}}) + cfg := TracesConfig{} + traces := GenerateTraces(cfg, &span, map[attr.Name]struct{}{attr.DBQueryText: {}}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -570,7 +590,8 @@ func TestGenerateTracesAttributes(t *testing.T) { }) t.Run("test Kafka trace generation", func(t *testing.T) { span := request.Span{Type: request.EventTypeKafkaClient, Method: "process", Path: "important-topic", OtherNamespace: "test"} - traces := GenerateTraces(&span, map[attr.Name]struct{}{}) + cfg := TracesConfig{} + traces := GenerateTraces(cfg, &span, map[attr.Name]struct{}{}) assert.Equal(t, 1, traces.ResourceSpans().Len()) assert.Equal(t, 1, traces.ResourceSpans().At(0).ScopeSpans().Len()) @@ -1143,7 +1164,8 @@ func generateTracesForSpans(t *testing.T, tr *tracesOTELReceiver, spans []reques if span.IgnoreSpan == request.IgnoreTraces || !tr.acceptSpan(span) { continue } - res = append(res, GenerateTraces(span, traceAttrs)) + cfg := TracesConfig{} + res = append(res, GenerateTraces(cfg, span, traceAttrs)) } return res diff --git a/pkg/internal/goexec/instructions.go b/pkg/internal/goexec/instructions.go index e96186170..4166888dd 100644 --- a/pkg/internal/goexec/instructions.go +++ b/pkg/internal/goexec/instructions.go @@ -12,9 +12,8 @@ import ( // instrumentationPoints loads the provided executable and looks for the addresses // where the start and return probes must be inserted. -// -//nolint:cyclop -func instrumentationPoints(elfF *elf.File, funcNames []string) (map[string]FuncOffsets, error) { +// nolint:cyclop +func instrumentationPoints(elfF *elf.File, funcNames []string) (map[string]FuncOffsets, *gosym.Table, error) { ilog := slog.With("component", "goexec.instructions") ilog.Debug("searching for instrumentation points", "functions", funcNames) functions := map[string]struct{}{} @@ -23,12 +22,12 @@ func instrumentationPoints(elfF *elf.File, funcNames []string) (map[string]FuncO } symTab, err := findGoSymbolTable(elfF) if err != nil { - return nil, err + return nil, nil, err } goVersion, _, err := getGoDetails(elfF) if err == nil && !supportedGoVersion(goVersion) { - return nil, fmt.Errorf("unsupported Go version: %v. Minimum supported version is %v", goVersion, minGoVersion) + return nil, nil, fmt.Errorf("unsupported Go version: %v. Minimum supported version is %v", goVersion, minGoVersion) } gosyms := elfF.Section(".gosymtab") @@ -40,7 +39,7 @@ func instrumentationPoints(elfF *elf.File, funcNames []string) (map[string]FuncO if gosyms == nil { allSyms, err = exec.FindExeSymbols(elfF, functions) if err != nil { - return nil, err + return nil, nil, err } } @@ -65,7 +64,7 @@ func instrumentationPoints(elfF *elf.File, funcNames []string) (map[string]FuncO offs, ok, err := findFuncOffset(&f, elfF) if err != nil { - return nil, err + return nil, nil, err } if ok { ilog.Debug("found relevant function for instrumentation", "function", fName, "offsets", offs) @@ -74,7 +73,7 @@ func instrumentationPoints(elfF *elf.File, funcNames []string) (map[string]FuncO } } - return allOffsets, nil + return allOffsets, symTab, nil } func handleStaticSymbol(fName string, allOffsets map[string]FuncOffsets, allSyms map[string]exec.Sym, ilog *slog.Logger) { diff --git a/pkg/internal/goexec/offsets.go b/pkg/internal/goexec/offsets.go index 525c05a4c..cc0834c11 100644 --- a/pkg/internal/goexec/offsets.go +++ b/pkg/internal/goexec/offsets.go @@ -2,15 +2,18 @@ package goexec import ( + "debug/gosym" "fmt" "github.com/grafana/beyla/pkg/internal/exec" + "github.com/grafana/beyla/pkg/internal/export/otel" ) type Offsets struct { // Funcs key: function name - Funcs map[string]FuncOffsets - Field FieldOffsets + Funcs map[string]FuncOffsets + Field FieldOffsets + SymTab *gosym.Table } type FuncOffsets struct { @@ -22,16 +25,22 @@ type FieldOffsets map[string]any // InspectOffsets gets the memory addresses/offsets of the instrumenting function, as well as the required // parameters fields to be read from the eBPF code -func InspectOffsets(execElf *exec.FileInfo, funcs []string) (*Offsets, error) { +func InspectOffsets(cfg *otel.TracesConfig, execElf *exec.FileInfo, funcs []string) (*Offsets, error) { if execElf == nil { return nil, fmt.Errorf("executable not found") } // Analyse executable ELF file and find instrumentation points - found, err := instrumentationPoints(execElf.ELF, funcs) + found, symTab, err := instrumentationPoints(execElf.ELF, funcs) if err != nil { return nil, fmt.Errorf("finding instrumentation points: %w", err) } + // symTab would be used to find the function name from the address when + // captuing Go errors. If the option is disabled, whe don't need to keep + // the symbol table in memory. + if !cfg.ReportExceptionEvents { + symTab = nil + } if len(found) == 0 { return nil, fmt.Errorf("couldn't find any instrumentation point in %s", execElf.CmdExePath) } @@ -43,7 +52,8 @@ func InspectOffsets(execElf *exec.FileInfo, funcs []string) (*Offsets, error) { } return &Offsets{ - Funcs: found, - Field: structFieldOffsets, + Funcs: found, + Field: structFieldOffsets, + SymTab: symTab, }, nil } diff --git a/pkg/internal/goexec/offsets_test.go b/pkg/internal/goexec/offsets_test.go index 5bfa88259..f7823c9e9 100644 --- a/pkg/internal/goexec/offsets_test.go +++ b/pkg/internal/goexec/offsets_test.go @@ -7,6 +7,7 @@ import ( "github.com/stretchr/testify/require" + "github.com/grafana/beyla/pkg/internal/export/otel" "github.com/grafana/beyla/pkg/internal/testutil" ) @@ -18,7 +19,8 @@ func TestProcessNotFound(t *testing.T) { finish := make(chan struct{}) go func() { defer close(finish) - _, err := InspectOffsets(nil, nil) + cfg := &otel.TracesConfig{} + _, err := InspectOffsets(cfg, nil, nil) require.Error(t, err) }() testutil.ReadChannel(t, finish, 5*time.Second) diff --git a/pkg/internal/pipe/instrumenter.go b/pkg/internal/pipe/instrumenter.go index db16ee878..96a9f152a 100644 --- a/pkg/internal/pipe/instrumenter.go +++ b/pkg/internal/pipe/instrumenter.go @@ -117,7 +117,7 @@ func newGraphBuilder(ctx context.Context, config *beyla.Config, ctxInfo *global. config.Traces.Grafana = &gb.config.Grafana.OTLP pipe.AddFinalProvider(gnb, otelTraces, otel.TracesReceiver(ctx, config.Traces, gb.ctxInfo, config.Attributes.Select)) pipe.AddFinalProvider(gnb, prometheus, prom.PrometheusEndpoint(ctx, gb.ctxInfo, &config.Prometheus, config.Attributes.Select)) - pipe.AddFinalProvider(gnb, alloyTraces, alloy.TracesReceiver(ctx, &config.TracesReceiver, config.Attributes.Select)) + pipe.AddFinalProvider(gnb, alloyTraces, alloy.TracesReceiver(ctx, config, config.Attributes.Select)) pipe.AddFinalProvider(gnb, noop, debug.NoopNode(config.Noop)) pipe.AddFinalProvider(gnb, printer, debug.PrinterNode(config.Printer)) diff --git a/pkg/internal/request/span.go b/pkg/internal/request/span.go index 84361e29c..8206fc672 100644 --- a/pkg/internal/request/span.go +++ b/pkg/internal/request/span.go @@ -63,30 +63,32 @@ type PidInfo struct { // REMINDER: any attribute here must be also added to the functions SpanOTELGetters, // SpanPromGetters and getDefinitions in pkg/internal/export/metric/definitions.go type Span struct { - Type EventType - IgnoreSpan IgnoreMode - Method string - Path string - Route string - Peer string - PeerPort int - Host string - HostPort int - Status int - ContentLength int64 - RequestStart int64 - Start int64 - End int64 - ServiceID svc.ID // TODO: rename to Service or ResourceAttrs - TraceID trace2.TraceID - SpanID trace2.SpanID - ParentSpanID trace2.SpanID - Flags uint8 - Pid PidInfo - PeerName string - HostName string - OtherNamespace string - Statement string + Type EventType + IgnoreSpan IgnoreMode + Method string + Path string + Route string + Peer string + PeerPort int + Host string + HostPort int + Status int + ContentLength int64 + RequestStart int64 + Start int64 + End int64 + ServiceID svc.ID // TODO: rename to Service or ResourceAttrs + TraceID trace2.TraceID + SpanID trace2.SpanID + ParentSpanID trace2.SpanID + Flags uint8 + Pid PidInfo + PeerName string + HostName string + OtherNamespace string + Statement string + ErrorMessage string + ErrorStacktrace string } func (s *Span) Inside(parent *Span) bool { diff --git a/test/integration/components/jaeger/jaeger.go b/test/integration/components/jaeger/jaeger.go index c3dfa19a1..3bf94d148 100644 --- a/test/integration/components/jaeger/jaeger.go +++ b/test/integration/components/jaeger/jaeger.go @@ -28,6 +28,7 @@ type Span struct { Duration int64 `json:"duration"` Tags []Tag `json:"tags"` ProcessID string `json:"processID"` + Logs []Log `json:"logs"` } type Tag struct { @@ -36,6 +37,11 @@ type Tag struct { Value interface{} `json:"value"` } +type Log struct { + Timestamp int64 `json:"timestamp"` + Fields []Tag `json:"fields"` +} + type Reference struct { RefType string `json:"refType"` TraceID string `json:"traceID"` diff --git a/test/integration/components/testserver/std/std.go b/test/integration/components/testserver/std/std.go index e47478704..6179e2f4f 100644 --- a/test/integration/components/testserver/std/std.go +++ b/test/integration/components/testserver/std/std.go @@ -54,6 +54,10 @@ func HTTPHandler(log *slog.Logger, echoPort int) http.HandlerFunc { } else { status = s } + if status == 500 { + echoError(rw) + return + } case arg.Delay: if d, err := time.ParseDuration(v[0]); err != nil { log.Debug("wrong delay value. Ignoring", "error", err) @@ -180,6 +184,12 @@ func echoCall(rw http.ResponseWriter) { rw.WriteHeader(204) } +func echoError(rw http.ResponseWriter) { + _, err := httpClient.Get("htt://pytestserver:8083/error") + slog.Error("error making http request", "err", err) + rw.WriteHeader(500) +} + func Setup(port int) { log := slog.With("component", "std.Server") address := fmt.Sprintf(":%d", port) diff --git a/test/integration/docker-compose.yml b/test/integration/docker-compose.yml index 082cfc559..38c527fbd 100644 --- a/test/integration/docker-compose.yml +++ b/test/integration/docker-compose.yml @@ -49,6 +49,7 @@ services: BEYLA_INTERNAL_METRICS_PROMETHEUS_PORT: 8999 BEYLA_PROCESSES_INTERVAL: "100ms" BEYLA_HOSTNAME: "beyla" + BEYLA_TRACES_REPORT_EXCEPTION_EVENTS: "true" ports: - "8999:8999" # Prometheus scrape port, if enabled via config diff --git a/test/integration/http2go_test.go b/test/integration/http2go_test.go index af8ff3613..2a99d6e0f 100644 --- a/test/integration/http2go_test.go +++ b/test/integration/http2go_test.go @@ -32,7 +32,7 @@ func testREDMetricsForHTTP2Library(t *testing.T, route, svcNs string) { `http_route="` + route + `",` + `url_path="` + route + `"}`) require.NoError(t, err) - // check duration_count has 3 calls and all the arguments + // check duration_count has 1 calls and all the arguments enoughPromResults(t, results) val := totalPromCount(t, results) assert.LessOrEqual(t, 1, val) diff --git a/test/integration/traces_test.go b/test/integration/traces_test.go index bc7c9a215..6cc8b77de 100644 --- a/test/integration/traces_test.go +++ b/test/integration/traces_test.go @@ -91,6 +91,17 @@ func testHTTPTracesCommon(t *testing.T, doTraceID bool, httpCode int) { jaeger.Tag{Key: "otel.status_code", Type: "string", Value: "ERROR"}, ) assert.Empty(t, sd, sd.String()) + assert.NotEmpty(t, parent.Logs) + eventAttr := parent.Logs[0].Fields + eventType, ok := jaeger.FindIn(eventAttr, "event") + require.True(t, ok) + assert.Equal(t, "exception", eventType.Value) + + _, ok = jaeger.FindIn(eventAttr, "exception.message") + require.True(t, ok) + + _, ok = jaeger.FindIn(eventAttr, "exception.stacktrace") + require.True(t, ok) } // Check the information of the "in queue" span