diff --git a/.gitignore b/.gitignore index fde737a2c..6fb1816da 100644 --- a/.gitignore +++ b/.gitignore @@ -7,4 +7,5 @@ ebpf-agent.tar *.pcap protoc/ release-assets/ +perf/ diff --git a/Dockerfile b/Dockerfile index 2662e64f7..10d3932f9 100644 --- a/Dockerfile +++ b/Dockerfile @@ -19,10 +19,15 @@ COPY go.sum go.sum # Build RUN CGO_ENABLED=0 GOARCH=$TARGETARCH go build -ldflags "$LDFLAGS" -mod vendor -a -o bin/netobserv-ebpf-agent cmd/netobserv-ebpf-agent.go +# Build collect-kernel-stats utility +# Note: CGO may be required for cilium/ebpf library interactions with kernel +RUN GOARCH=$TARGETARCH go build -mod vendor -a -o bin/collect-kernel-stats ./cmd/collect-kernel-stats + # Create final image from minimal + built binary FROM --platform=linux/$TARGETARCH registry.access.redhat.com/ubi9/ubi-minimal:9.6-1758184547 WORKDIR / COPY --from=builder /opt/app-root/bin/netobserv-ebpf-agent . +COPY --from=builder /opt/app-root/bin/collect-kernel-stats /collect-kernel-stats USER 65532:65532 ENTRYPOINT ["/netobserv-ebpf-agent"] diff --git a/Makefile b/Makefile index aa3ebd979..6894a37c7 100644 --- a/Makefile +++ b/Makefile @@ -159,6 +159,11 @@ compile: ## Compile ebpf agent project @echo "### Compiling project" GOARCH=${GOARCH} GOOS=$(GOOS) go build -mod vendor -o bin/netobserv-ebpf-agent cmd/netobserv-ebpf-agent.go +.PHONY: collect-kernel-stats +collect-kernel-stats: ## Build kernel-space eBPF statistics collector + @echo "### Building kernel stats collector" + GOARCH=${GOARCH} GOOS=$(GOOS) go build -mod vendor -o bin/collect-kernel-stats ./cmd/collect-kernel-stats + .PHONY: test test: ## Test code using go test @echo "### Testing code" diff --git a/bpf/flows.c b/bpf/flows.c index 6602c1904..e3371e78a 100644 --- a/bpf/flows.c +++ b/bpf/flows.c @@ -58,25 +58,78 @@ #include "ipsec.h" // return 0 on success, 1 if capacity reached +// Optimized: loop unrolled and early exits for common cases static __always_inline int add_observed_intf(flow_metrics *value, pkt_info *pkt, u32 if_index, u8 direction) { if (value->nb_observed_intf >= MAX_OBSERVED_INTERFACES) { return 1; } - for (u8 i = 0; i < value->nb_observed_intf; i++) { - if (value->observed_intf[i] == if_index) { - if (value->observed_direction[i] != direction && - value->observed_direction[i] != OBSERVED_DIRECTION_BOTH) { - // Same interface seen on a different direction => mark as both directions - value->observed_direction[i] = OBSERVED_DIRECTION_BOTH; - } - // Interface already seen -> skip - return 0; + + // Fast path: unroll loop for small array sizes (most common cases) + // Check each position explicitly to eliminate loop overhead + u8 nb = value->nb_observed_intf; + + // Unroll for common cases (0-3 interfaces) - most flows see 1-2 interfaces + if (nb == 0) { + // First interface - no check needed + goto add_new; + } + + // Check existing interfaces with unrolled comparisons + if (value->observed_intf[0] == if_index) { + if (value->observed_direction[0] != direction && + value->observed_direction[0] != OBSERVED_DIRECTION_BOTH) { + value->observed_direction[0] = OBSERVED_DIRECTION_BOTH; + } + return 0; + } + + if (nb >= 2 && value->observed_intf[1] == if_index) { + if (value->observed_direction[1] != direction && + value->observed_direction[1] != OBSERVED_DIRECTION_BOTH) { + value->observed_direction[1] = OBSERVED_DIRECTION_BOTH; } + return 0; } - value->observed_intf[value->nb_observed_intf] = if_index; - value->observed_direction[value->nb_observed_intf] = direction; - value->nb_observed_intf++; + + if (nb >= 3 && value->observed_intf[2] == if_index) { + if (value->observed_direction[2] != direction && + value->observed_direction[2] != OBSERVED_DIRECTION_BOTH) { + value->observed_direction[2] = OBSERVED_DIRECTION_BOTH; + } + return 0; + } + + // Fully unroll remaining cases (positions 3-5) for MAX_OBSERVED_INTERFACES=6 + if (nb >= 4 && value->observed_intf[3] == if_index) { + if (value->observed_direction[3] != direction && + value->observed_direction[3] != OBSERVED_DIRECTION_BOTH) { + value->observed_direction[3] = OBSERVED_DIRECTION_BOTH; + } + return 0; + } + + if (nb >= 5 && value->observed_intf[4] == if_index) { + if (value->observed_direction[4] != direction && + value->observed_direction[4] != OBSERVED_DIRECTION_BOTH) { + value->observed_direction[4] = OBSERVED_DIRECTION_BOTH; + } + return 0; + } + + if (nb >= 6 && value->observed_intf[5] == if_index) { + if (value->observed_direction[5] != direction && + value->observed_direction[5] != OBSERVED_DIRECTION_BOTH) { + value->observed_direction[5] = OBSERVED_DIRECTION_BOTH; + } + return 0; + } + +add_new: + // Not found - add new interface + value->observed_intf[nb] = if_index; + value->observed_direction[nb] = direction; + value->nb_observed_intf = nb + 1; return 0; } @@ -84,22 +137,34 @@ static __always_inline void update_existing_flow(flow_metrics *aggregate_flow, p u64 len, u32 sampling, u32 if_index, u8 direction) { // Count only packets seen from the same interface as previously to avoid duplicate counts + // Using lock-free atomic operations for better performance int maxReached = 0; - bpf_spin_lock(&aggregate_flow->lock); - if (aggregate_flow->if_index_first_seen == if_index) { - aggregate_flow->packets += 1; - aggregate_flow->bytes += len; + + // Read if_index_first_seen once (it's never modified after flow creation) + u32 first_seen = aggregate_flow->if_index_first_seen; + + if (first_seen == if_index) { + // Common path: same interface - use atomic operations + __sync_fetch_and_add(&aggregate_flow->packets, 1); + __sync_fetch_and_add(&aggregate_flow->bytes, len); + // Timestamp: use simple write (acceptable if slightly out of order, we want latest anyway) + // On architectures that support it, this will be naturally atomic for aligned 64-bit writes aggregate_flow->end_mono_time_ts = pkt->current_ts; + // Flags is u16 - eBPF doesn't support atomic ops on 16-bit types + // Use simple write: OR is idempotent, so worst case is missing a flag bit in rare races (acceptable) aggregate_flow->flags |= pkt->flags; + // DSCP and sampling: simple writes (these are infrequently updated, races are acceptable) aggregate_flow->dscp = pkt->dscp; aggregate_flow->sampling = sampling; } else if (if_index != 0) { - // Only add info that we've seen this interface (we can also update end time & flags) + // Different interface path: update timestamps/flags atomically, then add interface aggregate_flow->end_mono_time_ts = pkt->current_ts; + // Flags update - use simple write (OR is idempotent, occasional missed flag is acceptable) aggregate_flow->flags |= pkt->flags; + // Note: add_observed_intf may have races, but worst case is missing one interface entry + // This is acceptable since interface tracking is best-effort metadata maxReached = add_observed_intf(aggregate_flow, pkt, if_index, direction); } - bpf_spin_unlock(&aggregate_flow->lock); if (maxReached > 0) { BPF_PRINTK("observed interface missed (array capacity reached); ifindex=%d, eth_type=%d, " "proto=%d, sport=%d, dport=%d\n", @@ -138,25 +203,50 @@ static inline int flow_monitor(struct __sk_buff *skb, u8 direction) { } u16 eth_protocol = 0; + // Initialize pkt_info with only needed fields - compiler zeros the rest pkt_info pkt; - __builtin_memset(&pkt, 0, sizeof(pkt)); + pkt.current_ts = bpf_ktime_get_ns(); // Record the current time first. + pkt.id = NULL; // Will be set below + pkt.flags = 0; + pkt.l4_hdr = NULL; + pkt.dscp = 0; + pkt.dns_id = 0; + pkt.dns_flags = 0; + pkt.dns_latency = 0; + // DNS name only initialized if DNS tracking enabled (set by track_dns_packet if needed) - flow_id id; - __builtin_memset(&id, 0, sizeof(id)); + flow_id id = {0}; // All fields zeroed - needed for flow identification - pkt.current_ts = bpf_ktime_get_ns(); // Record the current time first. pkt.id = &id; void *data_end = (void *)(long)skb->data_end; void *data = (void *)(long)skb->data; struct ethhdr *eth = (struct ethhdr *)data; u64 len = skb->len; + u8 protocol = 0; // Will be set by L3 parsing - if (fill_ethhdr(eth, data_end, &pkt, ð_protocol) == DISCARD) { + // Optimized: Parse L2+L3 first for early IP filtering + // This allows us to skip L4 parsing if IP-based filtering rejects the packet + if (fill_ethhdr_l3only(eth, data_end, &pkt, ð_protocol, &protocol) == DISCARD) { return TC_ACT_OK; } - // check if this packet need to be filtered if filtering feature is enabled + // Early IP filtering: check if we can reject before parsing L4 + // This saves L4 parsing for packets that will be rejected anyway + bool filter_enabled = is_filter_enabled(); + if (filter_enabled) { + filter_action early_action = MAX_FILTER_ACTIONS; + if (early_ip_filter_check(&id, &early_action, eth_protocol, direction)) { + // Early rejection - skip L4 parsing entirely + if (early_action == REJECT) { + return TC_ACT_OK; + } + } + } + // Parse L4 (needed for full filtering or flow tracking) + parse_l4_after_l3(eth, data_end, &pkt, eth_protocol, protocol); + + // Full filter check (now that L4 is parsed if needed) bool skip = check_and_do_flow_filtering(&id, pkt.flags, 0, eth_protocol, &flow_sampling, direction); if (has_filter_sampling) { @@ -183,18 +273,20 @@ static inline int flow_monitor(struct __sk_buff *skb, u8 direction) { update_existing_flow(aggregate_flow, &pkt, len, flow_sampling, skb->ifindex, direction); } else { // Key does not exist in the map, and will need to create a new entry. - flow_metrics new_flow; - __builtin_memset(&new_flow, 0, sizeof(new_flow)); - new_flow.if_index_first_seen = skb->ifindex; - new_flow.direction_first_seen = direction; - new_flow.packets = 1; - new_flow.bytes = len; - new_flow.eth_protocol = eth_protocol; - new_flow.start_mono_time_ts = pkt.current_ts; - new_flow.end_mono_time_ts = pkt.current_ts; - new_flow.flags = pkt.flags; - new_flow.dscp = pkt.dscp; - new_flow.sampling = flow_sampling; + // Initialize only the fields we need - compiler will zero the rest + flow_metrics new_flow = { + .if_index_first_seen = skb->ifindex, + .direction_first_seen = direction, + .packets = 1, + .bytes = len, + .eth_protocol = eth_protocol, + .start_mono_time_ts = pkt.current_ts, + .end_mono_time_ts = pkt.current_ts, + .flags = pkt.flags, + .dscp = pkt.dscp, + .sampling = flow_sampling, + .nb_observed_intf = 0 // Explicitly zero for clarity + }; __builtin_memcpy(new_flow.dst_mac, eth->h_dest, ETH_ALEN); __builtin_memcpy(new_flow.src_mac, eth->h_source, ETH_ALEN); @@ -245,15 +337,17 @@ static inline int flow_monitor(struct __sk_buff *skb, u8 direction) { if (extra_metrics != NULL) { update_dns(extra_metrics, &pkt, dns_errno); } else { - additional_metrics new_metrics; - __builtin_memset(&new_metrics, 0, sizeof(new_metrics)); - new_metrics.start_mono_time_ts = pkt.current_ts; - new_metrics.end_mono_time_ts = pkt.current_ts; - new_metrics.eth_protocol = eth_protocol; - new_metrics.dns_record.id = pkt.dns_id; - new_metrics.dns_record.flags = pkt.dns_flags; - new_metrics.dns_record.latency = pkt.dns_latency; - new_metrics.dns_record.errno = dns_errno; + // Initialize only needed fields - compiler will zero the rest + additional_metrics new_metrics = { + .start_mono_time_ts = pkt.current_ts, + .end_mono_time_ts = pkt.current_ts, + .eth_protocol = eth_protocol, + .dns_record = {.id = pkt.dns_id, + .flags = pkt.dns_flags, + .latency = pkt.dns_latency, + .errno = dns_errno}, + .network_events_idx = 0 // Explicitly zero for clarity + }; long ret = bpf_map_update_elem(&additional_flow_metrics, &id, &new_metrics, BPF_NOEXIST); if (ret != 0) { diff --git a/bpf/flows_filter.h b/bpf/flows_filter.h index 22055c774..085da1da6 100644 --- a/bpf/flows_filter.h +++ b/bpf/flows_filter.h @@ -59,7 +59,7 @@ static __always_inline int do_flow_filter_lookup(flow_id *id, struct filter_key_ if (rule->do_peerCIDR_lookup) { struct filter_key_t peerKey; - __builtin_memset(&peerKey, 0, sizeof(peerKey)); + // No need to memset - flow_filter_setup_lookup_key will initialize all fields we use // PeerCIDR lookup will will target the opposite IP compared to original CIDR lookup // In other words if cidr is using srcIP then peerCIDR will be the dstIP if (flow_filter_setup_lookup_key(id, &peerKey, &len, &offset, use_src_ip, @@ -218,6 +218,62 @@ static __always_inline int do_flow_filter_lookup(flow_id *id, struct filter_key_ return result; } +/* + * Early IP-only filter check - optimized to skip L4 parsing if IP-based rejection is possible. + * Returns: 1 if packet can be rejected early (IP-only reject rule), 0 if needs full check + * This is a fast path that only checks CIDR matching, not ports/protocols. + */ +static __always_inline int early_ip_filter_check(flow_id *id, filter_action *action, + u16 eth_protocol, u8 direction) { + struct filter_key_t key; + u8 len, offset; + struct filter_value_t *rule; + + // Check srcIP CIDR match first + if (flow_filter_setup_lookup_key(id, &key, &len, &offset, true, eth_protocol) < 0) { + return 0; // Need full check + } + + rule = (struct filter_value_t *)bpf_map_lookup_elem(&filter_map, &key); + if (rule && rule->action == REJECT) { + // IP matches and action is REJECT - can reject early without checking ports/protocols + // Note: We check direction if rule specifies it + if (rule->direction == MAX_DIRECTION || rule->direction == direction) { + // If rule has port/protocol checks, we can't reject early (would need L4) + // But if it's IP-only (protocol==0, no ports), we can reject now + if (rule->protocol == 0 && rule->dstPortStart == 0 && rule->srcPortStart == 0 && + rule->portStart == 0 && rule->dstPort1 == 0 && rule->srcPort1 == 0 && + rule->port1 == 0 && rule->dstPort2 == 0 && rule->srcPort2 == 0 && + rule->port2 == 0 && !rule->do_peerCIDR_lookup && rule->tcpFlags == 0 && + rule->icmpType == 0 && rule->filter_drops == 0) { + *action = REJECT; + return 1; // Can reject early + } + } + } + + // Check dstIP CIDR match + if (flow_filter_setup_lookup_key(id, &key, &len, &offset, false, eth_protocol) < 0) { + return 0; // Need full check + } + + rule = (struct filter_value_t *)bpf_map_lookup_elem(&filter_map, &key); + if (rule && rule->action == REJECT) { + if (rule->direction == MAX_DIRECTION || rule->direction == direction) { + if (rule->protocol == 0 && rule->dstPortStart == 0 && rule->srcPortStart == 0 && + rule->portStart == 0 && rule->dstPort1 == 0 && rule->srcPort1 == 0 && + rule->port1 == 0 && rule->dstPort2 == 0 && rule->srcPort2 == 0 && + rule->port2 == 0 && !rule->do_peerCIDR_lookup && rule->tcpFlags == 0 && + rule->icmpType == 0 && rule->filter_drops == 0) { + *action = REJECT; + return 1; // Can reject early + } + } + } + + return 0; // Need full check with L4 +} + /* * check if the flow match filter rule and return >= 1 if the flow is to be dropped */ @@ -228,7 +284,7 @@ static __always_inline int is_flow_filtered(flow_id *id, filter_action *action, u8 len, offset; int result = 0; - __builtin_memset(&key, 0, sizeof(key)); + // No need to memset - flow_filter_setup_lookup_key will initialize all fields we use *action = MAX_FILTER_ACTIONS; // Lets do first CIDR match using srcIP. diff --git a/bpf/network_events_monitoring.h b/bpf/network_events_monitoring.h index 6adc099b7..3e6765170 100644 --- a/bpf/network_events_monitoring.h +++ b/bpf/network_events_monitoring.h @@ -7,11 +7,20 @@ #include "utils.h" +// Optimized: unroll loop for small array (MAX_NETWORK_EVENTS=4) static inline bool md_already_exists(u8 network_events[MAX_NETWORK_EVENTS][MAX_EVENT_MD], u8 *md) { - for (u8 i = 0; i < MAX_NETWORK_EVENTS; i++) { - if (__builtin_memcmp(network_events[i], md, MAX_EVENT_MD) == 0) { - return true; - } + // Unroll comparisons for common case - most flows have 1-2 events + if (__builtin_memcmp(network_events[0], md, MAX_EVENT_MD) == 0) { + return true; + } + if (__builtin_memcmp(network_events[1], md, MAX_EVENT_MD) == 0) { + return true; + } + if (__builtin_memcmp(network_events[2], md, MAX_EVENT_MD) == 0) { + return true; + } + if (__builtin_memcmp(network_events[3], md, MAX_EVENT_MD) == 0) { + return true; } return false; } diff --git a/bpf/types.h b/bpf/types.h index 500620de4..9a6965503 100644 --- a/bpf/types.h +++ b/bpf/types.h @@ -98,7 +98,7 @@ typedef struct flow_metrics_t { u8 dst_mac[ETH_ALEN]; // OS interface index u32 if_index_first_seen; - struct bpf_spin_lock lock; + // Lock removed - using lock-free atomic operations for better performance u32 sampling; u8 direction_first_seen; // The positive errno of a failed map insertion that caused a flow diff --git a/bpf/utils.h b/bpf/utils.h index 09a6b8533..bc46b9c20 100644 --- a/bpf/utils.h +++ b/bpf/utils.h @@ -109,6 +109,24 @@ static inline u8 ipv6_get_dscp(const struct ipv6hdr *ipv6h) { return ((bpf_ntohs(*(const __be16 *)ipv6h) >> 4) >> DSCP_SHIFT) & DSCP_MASK; } +// sets flow fields from IPv4 header information (L3 only, no L4) +// Returns protocol in *protocol for later L4 parsing +static inline int fill_iphdr_l3only(struct iphdr *ip, void *data_end, pkt_info *pkt, u8 *protocol) { + if ((void *)ip + sizeof(*ip) > data_end) { + return DISCARD; + } + flow_id *id = pkt->id; + /* Save the IP Address to id directly. copy once. */ + __builtin_memcpy(id->src_ip, ip4in6, sizeof(ip4in6)); + __builtin_memcpy(id->dst_ip, ip4in6, sizeof(ip4in6)); + __builtin_memcpy(id->src_ip + sizeof(ip4in6), &ip->saddr, sizeof(ip->saddr)); + __builtin_memcpy(id->dst_ip + sizeof(ip4in6), &ip->daddr, sizeof(ip->daddr)); + pkt->dscp = ipv4_get_dscp(ip); + *protocol = ip->protocol; // Store for later L4 parsing + // Note: L4 not parsed yet - will be parsed later if needed + return SUBMIT; +} + // sets flow fields from IPv4 header information static inline int fill_iphdr(struct iphdr *ip, void *data_end, pkt_info *pkt) { void *l4_hdr_start; @@ -129,6 +147,23 @@ static inline int fill_iphdr(struct iphdr *ip, void *data_end, pkt_info *pkt) { return SUBMIT; } +// sets flow fields from IPv6 header information (L3 only, no L4) +// Returns protocol in *protocol for later L4 parsing +static inline int fill_ip6hdr_l3only(struct ipv6hdr *ip, void *data_end, pkt_info *pkt, + u8 *protocol) { + if ((void *)ip + sizeof(*ip) > data_end) { + return DISCARD; + } + flow_id *id = pkt->id; + /* Save the IP Address to id directly. copy once. */ + __builtin_memcpy(id->src_ip, ip->saddr.in6_u.u6_addr8, IP_MAX_LEN); + __builtin_memcpy(id->dst_ip, ip->daddr.in6_u.u6_addr8, IP_MAX_LEN); + pkt->dscp = ipv6_get_dscp(ip); + *protocol = ip->nexthdr; // Store for later L4 parsing + // Note: L4 not parsed yet - will be parsed later if needed + return SUBMIT; +} + // sets flow fields from IPv6 header information static inline int fill_ip6hdr(struct ipv6hdr *ip, void *data_end, pkt_info *pkt) { void *l4_hdr_start; @@ -147,6 +182,48 @@ static inline int fill_ip6hdr(struct ipv6hdr *ip, void *data_end, pkt_info *pkt) return SUBMIT; } +// Parse L4 header separately (after L3 has been parsed) +// Optimized for early IP filtering flow +static inline void parse_l4_after_l3(struct ethhdr *eth, void *data_end, pkt_info *pkt, + u16 eth_protocol, u8 protocol) { + void *l4_hdr_start; + + if (eth_protocol == ETH_P_IP) { + struct iphdr *ip = (void *)eth + sizeof(*eth); + l4_hdr_start = (void *)ip + sizeof(*ip); + } else if (eth_protocol == ETH_P_IPV6) { + struct ipv6hdr *ip6 = (void *)eth + sizeof(*eth); + l4_hdr_start = (void *)ip6 + sizeof(*ip6); + } else { + return; // Not IP, shouldn't happen + } + + if (l4_hdr_start <= data_end) { + fill_l4info(l4_hdr_start, data_end, protocol, pkt); + } +} + +// sets flow fields from Ethernet header information (L2+L3 only, no L4) +// Optimized for early IP filtering - skips L4 parsing +// Returns protocol in *protocol for later L4 parsing +static inline int fill_ethhdr_l3only(struct ethhdr *eth, void *data_end, pkt_info *pkt, + u16 *eth_protocol, u8 *protocol) { + if ((void *)eth + sizeof(*eth) > data_end) { + return DISCARD; + } + *eth_protocol = bpf_ntohs(eth->h_proto); + + if (*eth_protocol == ETH_P_IP) { + struct iphdr *ip = (void *)eth + sizeof(*eth); + return fill_iphdr_l3only(ip, data_end, pkt, protocol); + } else if (*eth_protocol == ETH_P_IPV6) { + struct ipv6hdr *ip6 = (void *)eth + sizeof(*eth); + return fill_ip6hdr_l3only(ip6, data_end, pkt, protocol); + } + // Only IP-based flows are managed + return DISCARD; +} + // sets flow fields from Ethernet header information static inline int fill_ethhdr(struct ethhdr *eth, void *data_end, pkt_info *pkt, u16 *eth_protocol) { diff --git a/cmd/collect-kernel-stats.go b/cmd/collect-kernel-stats.go new file mode 100644 index 000000000..ee9753c5a --- /dev/null +++ b/cmd/collect-kernel-stats.go @@ -0,0 +1,243 @@ +//go:build linux + +package main + +import ( + "encoding/json" + "errors" + "flag" + "fmt" + "os" + "time" + + "github.com/cilium/ebpf" + "github.com/sirupsen/logrus" + "golang.org/x/sys/unix" +) + +type KernelStats struct { + Timestamp time.Time `json:"timestamp"` + CollectionDuration time.Duration `json:"collection_duration_sec"` + ProgramStats map[string]ProgramKernelStats `json:"program_stats"` + TotalRuntime time.Duration `json:"total_runtime_ns"` + TotalRunCount uint64 `json:"total_run_count"` + TotalEventsPerSec float64 `json:"total_events_per_sec"` + TotalEstimatedCPU float64 `json:"total_estimated_cpu_percent"` + TotalRecursionMisses uint64 `json:"total_recursion_misses"` + NumPrograms int `json:"num_programs"` +} + +type ProgramKernelStats struct { + Name string `json:"name"` + Type string `json:"type"` + ID ebpf.ProgramID `json:"id"` + Tag string `json:"tag,omitempty"` // BPF bytecode hash (available from 4.13) + Runtime time.Duration `json:"runtime_ns"` + RunCount uint64 `json:"run_count"` + EventsPerSec float64 `json:"events_per_sec"` + AvgRuntime time.Duration `json:"avg_runtime_ns"` + EstimatedCPU float64 `json:"estimated_cpu_percent"` + RecursionMisses uint64 `json:"recursion_misses"` + Memlock uint64 `json:"memlock_bytes,omitempty"` // Memory locked by program (available from 4.10) + JitedSize uint32 `json:"jited_size_bytes,omitempty"` // JIT-compiled code size (available from 4.13) + VerifiedInstructions uint32 `json:"verified_instructions,omitempty"` // Number of verified instructions (available from 5.16) + NumMaps int `json:"num_maps,omitempty"` // Number of associated maps (available from 4.15) +} + +func collectKernelStats(duration time.Duration, interval time.Duration) (*KernelStats, error) { + // Enable BPF statistics collection + closer, err := ebpf.EnableStats(unix.BPF_STATS_RUN_TIME) + if err != nil { + return nil, fmt.Errorf("failed to enable BPF stats: %w", err) + } + defer closer.Close() + + logrus.Infof("Collecting kernel-space eBPF statistics for %v (sampling every %v)", duration, interval) + + // Collect initial stats + initialStats := make(map[ebpf.ProgramID]ProgramKernelStats) + programInfo := make(map[ebpf.ProgramID]*ebpf.ProgramInfo) + + // Iterate over all loaded programs + var progID ebpf.ProgramID = 0 + for { + nextID, err := ebpf.ProgramGetNextID(progID) + if err != nil { + // ErrNotExist means no more programs, which is expected + if errors.Is(err, os.ErrNotExist) || errors.Is(err, ebpf.ErrNotSupported) { + break + } + // Other errors are logged but we continue + logrus.WithError(err).Warn("error getting next program ID") + break + } + progID = nextID + prog, err := ebpf.NewProgramFromID(progID) + if err != nil { + logrus.WithError(err).WithField("program_id", progID).Warn("failed to open program") + continue + } + + info, err := prog.Info() + if err != nil { + prog.Close() + logrus.WithError(err).WithField("program_id", progID).Warn("failed to get program info") + continue + } + + stats, err := prog.Stats() + if err != nil { + prog.Close() + logrus.WithError(err).WithField("program_id", progID).Warn("failed to get program stats") + continue + } + + programInfo[progID] = info + + // Extract additional metadata + memlock, _ := info.Memlock() + jitedSize, _ := info.JitedSize() + verifiedInsns, _ := info.VerifiedInstructions() + mapIDs, _ := info.MapIDs() + + initialStats[progID] = ProgramKernelStats{ + Name: info.Name, + Type: info.Type.String(), + ID: progID, + Tag: info.Tag, + Runtime: stats.Runtime, + RunCount: stats.RunCount, + RecursionMisses: stats.RecursionMisses, + Memlock: memlock, + JitedSize: jitedSize, + VerifiedInstructions: verifiedInsns, + NumMaps: len(mapIDs), + } + + prog.Close() + } + + if len(initialStats) == 0 { + return nil, fmt.Errorf("no eBPF programs found") + } + + logrus.Infof("Found %d eBPF programs, collecting stats for %v", len(initialStats), duration) + + // Wait for collection duration + time.Sleep(duration) + + // Collect final stats and calculate metrics + finalStats := make(map[string]ProgramKernelStats) + var totalRuntime time.Duration + var totalRunCount uint64 + + for progID, initialStat := range initialStats { + prog, err := ebpf.NewProgramFromID(progID) + if err != nil { + continue + } + + stats, err := prog.Stats() + if err != nil { + prog.Close() + continue + } + + info := programInfo[progID] + deltaRuntime := stats.Runtime - initialStat.Runtime + deltaRunCount := stats.RunCount - initialStat.RunCount + + var eventsPerSec float64 + var avgRuntime time.Duration + var estimatedCPU float64 + + if duration > 0 { + eventsPerSec = float64(deltaRunCount) / duration.Seconds() + if deltaRunCount > 0 { + avgRuntime = deltaRuntime / time.Duration(deltaRunCount) + // Estimated CPU % = (total runtime / collection duration) * 100 + estimatedCPU = (float64(deltaRuntime) / duration.Seconds()) / 1e9 * 100 + } + } + + // Extract additional metadata (these don't change during runtime) + memlock, _ := info.Memlock() + jitedSize, _ := info.JitedSize() + verifiedInsns, _ := info.VerifiedInstructions() + mapIDs, _ := info.MapIDs() + + finalStats[info.Name] = ProgramKernelStats{ + Name: info.Name, + Type: info.Type.String(), + ID: progID, + Tag: info.Tag, + Runtime: deltaRuntime, + RunCount: deltaRunCount, + EventsPerSec: eventsPerSec, + AvgRuntime: avgRuntime, + EstimatedCPU: estimatedCPU, + RecursionMisses: stats.RecursionMisses - initialStat.RecursionMisses, + Memlock: memlock, + JitedSize: jitedSize, + VerifiedInstructions: verifiedInsns, + NumMaps: len(mapIDs), + } + + totalRuntime += deltaRuntime + totalRunCount += deltaRunCount + + prog.Close() + } + + var totalEventsPerSec float64 + var totalEstimatedCPU float64 + if duration > 0 { + totalEventsPerSec = float64(totalRunCount) / duration.Seconds() + totalEstimatedCPU = (float64(totalRuntime) / duration.Seconds()) / 1e9 * 100 + } + + return &KernelStats{ + Timestamp: time.Now(), + ProgramStats: finalStats, + TotalRuntime: totalRuntime, + TotalRunCount: totalRunCount, + TotalEventsPerSec: totalEventsPerSec, + TotalEstimatedCPU: totalEstimatedCPU, + NumPrograms: len(finalStats), + }, nil +} + +func main() { + var ( + duration = flag.Duration("duration", 60*time.Second, "Duration to collect statistics") + interval = flag.Duration("interval", 1*time.Second, "Sampling interval (not used, kept for compatibility)") + output = flag.String("output", "", "Output JSON file (default: stdout)") + verbose = flag.Bool("verbose", false, "Enable verbose logging") + ) + flag.Parse() + + if *verbose { + logrus.SetLevel(logrus.DebugLevel) + } else { + logrus.SetLevel(logrus.InfoLevel) + } + + stats, err := collectKernelStats(*duration, *interval) + if err != nil { + logrus.WithError(err).Fatal("Failed to collect kernel statistics") + } + + jsonData, err := json.MarshalIndent(stats, "", " ") + if err != nil { + logrus.WithError(err).Fatal("Failed to marshal statistics to JSON") + } + + if *output != "" { + if err := os.WriteFile(*output, jsonData, 0644); err != nil { + logrus.WithError(err).Fatal("Failed to write output file") + } + logrus.Infof("Kernel statistics saved to %s", *output) + } else { + fmt.Println(string(jsonData)) + } +} diff --git a/cmd/collect-kernel-stats/README.md b/cmd/collect-kernel-stats/README.md new file mode 100644 index 000000000..bc896568e --- /dev/null +++ b/cmd/collect-kernel-stats/README.md @@ -0,0 +1,220 @@ +# Kernel-Space eBPF Statistics Collector + +This tool collects kernel-space performance statistics for running eBPF programs, similar to [bpftop](https://github.com/Netflix/bpftop). It uses the `BPF_ENABLE_STATS` syscall to gather runtime statistics directly from the kernel. + +## Overview + +Unlike user-space metrics (CPU, memory usage of the userspace agent), kernel-space metrics capture the actual performance of eBPF programs running in the kernel: + +- **Runtime**: Total time spent executing eBPF programs +- **Run Count**: Number of times programs executed +- **Events Per Second**: Rate of program executions +- **Average Runtime**: Average execution time per program run +- **Estimated CPU %**: Estimated CPU percentage used by eBPF programs +- **Recursion Misses**: Number of times programs couldn't run due to recursion limits + +## Building + +```bash +go build -o collect-kernel-stats ./cmd/collect-kernel-stats +``` + +Or using make (add to Makefile): + +```bash +make collect-kernel-stats +``` + +## Usage + +### Basic Usage + +Collect statistics for 60 seconds (default): + +```bash +sudo ./collect-kernel-stats +``` + +### Options + +```bash +./collect-kernel-stats [OPTIONS] + +Options: + -duration duration Duration to collect statistics (default: 60s) + -output string Output JSON file (default: stdout) + -verbose Enable verbose logging +``` + +### Examples + +```bash +# Collect for 5 minutes and save to file +sudo ./collect-kernel-stats -duration 5m -output kernel-stats.json + +# Collect for 30 seconds and print to stdout +sudo ./collect-kernel-stats -duration 30s + +# Verbose output +sudo ./collect-kernel-stats -duration 2m -verbose +``` + +## Requirements + +- **Root privileges**: Must run with `sudo` (required for `BPF_ENABLE_STATS` syscall) +- **Linux kernel 5.8+**: Required for eBPF statistics support +- **eBPF programs loaded**: The tool will only collect stats if eBPF programs are currently loaded + +## Output Format + +The tool outputs JSON with the following structure: + +```json +{ + "timestamp": "2024-01-15T10:30:00Z", + "program_stats": { + "tc_ingress_flow_parse": { + "name": "tc_ingress_flow_parse", + "type": "SchedCLS", + "id": 42, + "runtime_ns": 1234567890, + "run_count": 1000000, + "events_per_sec": 16666.67, + "avg_runtime_ns": 1234, + "estimated_cpu_percent": 1.23, + "recursion_misses": 0 + }, + ... + }, + "total_runtime_ns": 9876543210, + "total_run_count": 5000000, + "total_events_per_sec": 83333.33, + "total_estimated_cpu_percent": 9.88, + "num_programs": 8 +} +``` + +## Integration with Performance Tests + +### Automatic Collection During e2e Tests + +Kernel stats are **automatically collected** during e2e tests when you run: + +```bash +make tests-e2e +``` + +The kernel stats will be saved to `e2e-logs/{cluster-name}/kernel-stats.json` after the tests complete. + +### Manual Collection + +To collect kernel-space metrics manually during performance tests: + +1. **Start the eBPF agent** (which loads the eBPF programs) +2. **Run the performance test** (generate traffic) +3. **Collect kernel stats** in parallel: + +```bash +sudo ./bin/collect-kernel-stats -duration 5m -output kernel-stats-$(date +%s).json +``` + +### Baseline Comparison + +You can automatically compare kernel stats against the main branch: + +```bash +make tests-e2e BASELINE=true +``` + +This will: +- Run e2e tests with main branch image and collect baseline stats +- Run e2e tests with your branch image and collect current stats +- Generate a comparison visualization at `perf/kernel-comparison-main.png` + +See [e2e/README.md](../../e2e/README.md) for more details. + +## Integration with Visualization + +The kernel stats can be integrated into the performance visualization by: + +1. Collecting kernel stats during performance runs (automatically during e2e tests) +2. Storing the JSON output alongside CSV performance data +3. Using the visualization script with kernel stats support (see `scripts/visualize_ebpf_performance.py`) + +**Examples:** + +```bash +# Visualize kernel stats only +python3 scripts/visualize_ebpf_performance.py --kernel-stats e2e-logs/*/kernel-stats.json + +# Compare with baseline (main branch) +python3 scripts/visualize_ebpf_performance.py \ + --kernel-stats perf/kernel-stats-current.json \ + --kernel-stats-baseline perf/kernel-stats-main.json \ + --output perf/comparison.png +``` + +## How It Works + +1. **Enable Statistics**: Uses `ebpf.EnableStats(unix.BPF_STATS_RUN_TIME)` to enable kernel statistics collection +2. **Discover Programs**: Iterates through all loaded eBPF programs using `ebpf.ProgramGetNextID()` - captures all program IDs +3. **Collect Baseline**: Records initial stats for each program +4. **Wait**: Collects statistics for the specified duration +5. **Collect Final**: Records final stats and calculates deltas (runtime, run count, etc.) +6. **Calculate Metrics**: Computes events/sec, average runtime, and estimated CPU usage +7. **Generate Report**: Outputs JSON with statistics for all collected programs + +## Program IDs + +The tool collects statistics for **all** loaded eBPF programs in the system. Each eBPF program has a unique **program ID** assigned by the kernel when loaded. + +### How Program IDs Work + +- Each eBPF program gets a unique **program ID** assigned by the kernel when loaded +- Program IDs are assigned sequentially as programs are loaded +- The tool captures all program IDs during collection and includes them in the output + +### Viewing Program IDs + +To see all program IDs collected: + +```bash +# Quick 1-second collection to see all program IDs +sudo ./collect-kernel-stats -duration 1s | jq '.program_stats[].id' + +# Or extract min/max IDs +MIN_ID=$(sudo ./collect-kernel-stats -duration 1s | jq '[.program_stats[].id] | min') +MAX_ID=$(sudo ./collect-kernel-stats -duration 1s | jq '[.program_stats[].id] | max') +``` + +### Multiple Agents in Same Cluster + +When multiple eBPF agents run in the same cluster (e.g., for benchmark comparisons), the tool collects statistics for **all** loaded programs from all agents. This allows for comprehensive comparison of performance across different agent versions running simultaneously. + +## Comparison with bpftop + +This tool is similar to Netflix's `bpftop` but: +- **Outputs JSON** for easy integration with automation +- **Non-interactive** by default (suitable for CI/CD) +- **Built with Go** using the same cilium/ebpf library as the agent +- **Focuses on aggregation** over a collection period rather than real-time display + +## Troubleshooting + +**"No eBPF programs found"** +- Ensure the eBPF agent is running and has loaded programs +- Check that programs are loaded: `sudo bpftool prog list` + +**"Failed to enable BPF stats"** +- Requires Linux kernel 5.8+ with `CONFIG_BPF_STATS` enabled +- Must run with root privileges + +**"Permission denied"** +- The tool requires `sudo` to access kernel eBPF statistics + +## References + +- [bpftop](https://github.com/Netflix/bpftop) - Netflix's interactive eBPF monitoring tool +- [BPF_ENABLE_STATS documentation](https://www.kernel.org/doc/html/latest/bpf/bpf_stats.html) +- [cilium/ebpf library](https://github.com/cilium/ebpf) + diff --git a/cmd/collect-kernel-stats/main.go b/cmd/collect-kernel-stats/main.go new file mode 100644 index 000000000..a5ecc363e --- /dev/null +++ b/cmd/collect-kernel-stats/main.go @@ -0,0 +1,226 @@ +//go:build linux + +package main + +import ( + "encoding/json" + "errors" + "flag" + "fmt" + "os" + "time" + + "github.com/cilium/ebpf" + "github.com/sirupsen/logrus" + "golang.org/x/sys/unix" +) + +type KernelStats struct { + Timestamp time.Time `json:"timestamp"` + CollectionDuration time.Duration `json:"collection_duration_ns"` + ProgramStats map[string]ProgramKernelStats `json:"program_stats"` + TotalRuntime time.Duration `json:"total_runtime_ns"` + TotalRunCount uint64 `json:"total_run_count"` + TotalEventsPerSec float64 `json:"total_events_per_sec"` + TotalEstimatedCPU float64 `json:"total_estimated_cpu_percent"` + TotalRecursionMisses uint64 `json:"total_recursion_misses"` + NumPrograms int `json:"num_programs"` +} + +type ProgramKernelStats struct { + Name string `json:"name"` + Type string `json:"type"` + ID ebpf.ProgramID `json:"id"` + Runtime time.Duration `json:"runtime_ns"` + RunCount uint64 `json:"run_count"` + EventsPerSec float64 `json:"events_per_sec"` + AvgRuntime time.Duration `json:"avg_runtime_ns"` + EstimatedCPU float64 `json:"estimated_cpu_percent"` + RecursionMisses uint64 `json:"recursion_misses"` +} + +func collectKernelStats(duration time.Duration) (*KernelStats, error) { + // Enable BPF statistics collection + closer, err := ebpf.EnableStats(unix.BPF_STATS_RUN_TIME) + if err != nil { + return nil, fmt.Errorf("failed to enable BPF stats: %w", err) + } + defer closer.Close() + + logrus.Infof("Collecting kernel-space eBPF statistics for %v", duration) + + // Collect initial stats for all programs + initialStats := make(map[ebpf.ProgramID]ProgramKernelStats) + programInfo := make(map[ebpf.ProgramID]*ebpf.ProgramInfo) + + // Iterate over all loaded programs + var progID ebpf.ProgramID = 0 + for { + nextID, err := ebpf.ProgramGetNextID(progID) + if err != nil { + // ErrNotExist means no more programs, which is expected + if errors.Is(err, os.ErrNotExist) || errors.Is(err, ebpf.ErrNotSupported) { + break + } + // Other errors are logged but we continue + logrus.WithError(err).Warn("error getting next program ID") + break + } + progID = nextID + + // Collect stats for all programs + prog, err := ebpf.NewProgramFromID(progID) + if err != nil { + logrus.WithError(err).WithField("program_id", progID).Warn("failed to open program") + continue + } + + info, err := prog.Info() + if err != nil { + prog.Close() + logrus.WithError(err).WithField("program_id", progID).Warn("failed to get program info") + continue + } + + stats, err := prog.Stats() + if err != nil { + prog.Close() + logrus.WithError(err).WithField("program_id", progID).Warn("failed to get program stats") + continue + } + + programInfo[progID] = info + initialStats[progID] = ProgramKernelStats{ + Name: info.Name, + Type: info.Type.String(), + ID: progID, + Runtime: stats.Runtime, + RunCount: stats.RunCount, + RecursionMisses: stats.RecursionMisses, + } + + prog.Close() + } + + if len(initialStats) == 0 { + return nil, fmt.Errorf("no eBPF programs found") + } + + logrus.Infof("Found %d eBPF programs, collecting stats for %v", len(initialStats), duration) + + // Wait for collection duration + time.Sleep(duration) + + // Collect final stats and calculate metrics + finalStats := make(map[string]ProgramKernelStats) + var totalRuntime time.Duration + var totalRunCount uint64 + var totalRecursionMisses uint64 + + for progID, initialStat := range initialStats { + prog, err := ebpf.NewProgramFromID(progID) + if err != nil { + continue + } + + stats, err := prog.Stats() + if err != nil { + prog.Close() + continue + } + + info := programInfo[progID] + deltaRuntime := stats.Runtime - initialStat.Runtime + deltaRunCount := stats.RunCount - initialStat.RunCount + deltaRecursionMisses := stats.RecursionMisses - initialStat.RecursionMisses + + var eventsPerSec float64 + var avgRuntime time.Duration + var estimatedCPU float64 + + if duration > 0 { + eventsPerSec = float64(deltaRunCount) / duration.Seconds() + if deltaRunCount > 0 && deltaRuntime >= 0 { + avgRuntime = deltaRuntime / time.Duration(deltaRunCount) + // Estimated CPU % = (total runtime / collection duration) * 100 + // deltaRuntime is in nanoseconds, duration.Seconds() is in seconds + // Divide by 1e9 to convert ns to seconds, then multiply by 100 for percentage + estimatedCPU = (float64(deltaRuntime) / duration.Seconds()) / 1e9 * 100 + } + } + + finalStats[info.Name] = ProgramKernelStats{ + Name: info.Name, + Type: info.Type.String(), + ID: progID, + Runtime: deltaRuntime, + RunCount: deltaRunCount, + EventsPerSec: eventsPerSec, + AvgRuntime: avgRuntime, + EstimatedCPU: estimatedCPU, + RecursionMisses: deltaRecursionMisses, + } + + totalRuntime += deltaRuntime + totalRunCount += deltaRunCount + totalRecursionMisses += deltaRecursionMisses + + prog.Close() + } + + var totalEventsPerSec float64 + var totalEstimatedCPU float64 + if duration > 0 { + totalEventsPerSec = float64(totalRunCount) / duration.Seconds() + // Only calculate CPU if we have positive runtime (negative would indicate counter issues) + if totalRuntime >= 0 { + totalEstimatedCPU = (float64(totalRuntime) / duration.Seconds()) / 1e9 * 100 + } + } + + return &KernelStats{ + Timestamp: time.Now(), + CollectionDuration: duration, + ProgramStats: finalStats, + TotalRuntime: totalRuntime, + TotalRunCount: totalRunCount, + TotalEventsPerSec: totalEventsPerSec, + TotalEstimatedCPU: totalEstimatedCPU, + TotalRecursionMisses: totalRecursionMisses, + NumPrograms: len(finalStats), + }, nil +} + +func main() { + var ( + duration = flag.Duration("duration", 60*time.Second, "Duration to collect statistics") + output = flag.String("output", "", "Output JSON file (default: stdout)") + verbose = flag.Bool("verbose", false, "Enable verbose logging") + ) + flag.Parse() + + if *verbose { + logrus.SetLevel(logrus.DebugLevel) + } else { + logrus.SetLevel(logrus.InfoLevel) + } + + stats, err := collectKernelStats(*duration) + if err != nil { + logrus.WithError(err).Fatal("Failed to collect kernel statistics") + } + + jsonData, err := json.MarshalIndent(stats, "", " ") + if err != nil { + logrus.WithError(err).Fatal("Failed to marshal statistics to JSON") + } + + if *output != "" { + if err := os.WriteFile(*output, jsonData, 0644); err != nil { + logrus.WithError(err).Fatal("Failed to write output file") + } + logrus.Infof("Kernel statistics saved to %s", *output) + } else { + fmt.Println(string(jsonData)) + } +} diff --git a/pkg/ebpf/bpf_arm64_bpfel.go b/pkg/ebpf/bpf_arm64_bpfel.go index 400595e5e..265af46bd 100644 --- a/pkg/ebpf/bpf_arm64_bpfel.go +++ b/pkg/ebpf/bpf_arm64_bpfel.go @@ -116,20 +116,16 @@ type BpfFlowIdT struct { type BpfFlowMetrics BpfFlowMetricsT type BpfFlowMetricsT struct { - _ structs.HostLayout - StartMonoTimeTs uint64 - EndMonoTimeTs uint64 - Bytes uint64 - Packets uint32 - EthProtocol uint16 - Flags uint16 - SrcMac [6]uint8 - DstMac [6]uint8 - IfIndexFirstSeen uint32 - Lock struct { - _ structs.HostLayout - Val uint32 - } + _ structs.HostLayout + StartMonoTimeTs uint64 + EndMonoTimeTs uint64 + Bytes uint64 + Packets uint32 + EthProtocol uint16 + Flags uint16 + SrcMac [6]uint8 + DstMac [6]uint8 + IfIndexFirstSeen uint32 Sampling uint32 DirectionFirstSeen uint8 Errno uint8 @@ -138,7 +134,6 @@ type BpfFlowMetricsT struct { ObservedDirection [6]uint8 _ [2]byte ObservedIntf [6]uint32 - _ [4]byte } type BpfFlowRecordT struct { diff --git a/pkg/ebpf/bpf_arm64_bpfel.o b/pkg/ebpf/bpf_arm64_bpfel.o index 54d7d1d9d..7b497cae9 100644 Binary files a/pkg/ebpf/bpf_arm64_bpfel.o and b/pkg/ebpf/bpf_arm64_bpfel.o differ diff --git a/pkg/ebpf/bpf_powerpc_bpfel.go b/pkg/ebpf/bpf_powerpc_bpfel.go index dd023874b..6811a67db 100644 --- a/pkg/ebpf/bpf_powerpc_bpfel.go +++ b/pkg/ebpf/bpf_powerpc_bpfel.go @@ -116,20 +116,16 @@ type BpfFlowIdT struct { type BpfFlowMetrics BpfFlowMetricsT type BpfFlowMetricsT struct { - _ structs.HostLayout - StartMonoTimeTs uint64 - EndMonoTimeTs uint64 - Bytes uint64 - Packets uint32 - EthProtocol uint16 - Flags uint16 - SrcMac [6]uint8 - DstMac [6]uint8 - IfIndexFirstSeen uint32 - Lock struct { - _ structs.HostLayout - Val uint32 - } + _ structs.HostLayout + StartMonoTimeTs uint64 + EndMonoTimeTs uint64 + Bytes uint64 + Packets uint32 + EthProtocol uint16 + Flags uint16 + SrcMac [6]uint8 + DstMac [6]uint8 + IfIndexFirstSeen uint32 Sampling uint32 DirectionFirstSeen uint8 Errno uint8 @@ -138,7 +134,6 @@ type BpfFlowMetricsT struct { ObservedDirection [6]uint8 _ [2]byte ObservedIntf [6]uint32 - _ [4]byte } type BpfFlowRecordT struct { diff --git a/pkg/ebpf/bpf_powerpc_bpfel.o b/pkg/ebpf/bpf_powerpc_bpfel.o index ae6a7fe2c..6c702c4a0 100644 Binary files a/pkg/ebpf/bpf_powerpc_bpfel.o and b/pkg/ebpf/bpf_powerpc_bpfel.o differ diff --git a/pkg/ebpf/bpf_s390_bpfeb.go b/pkg/ebpf/bpf_s390_bpfeb.go index fc25078be..d3a96551c 100644 --- a/pkg/ebpf/bpf_s390_bpfeb.go +++ b/pkg/ebpf/bpf_s390_bpfeb.go @@ -116,20 +116,16 @@ type BpfFlowIdT struct { type BpfFlowMetrics BpfFlowMetricsT type BpfFlowMetricsT struct { - _ structs.HostLayout - StartMonoTimeTs uint64 - EndMonoTimeTs uint64 - Bytes uint64 - Packets uint32 - EthProtocol uint16 - Flags uint16 - SrcMac [6]uint8 - DstMac [6]uint8 - IfIndexFirstSeen uint32 - Lock struct { - _ structs.HostLayout - Val uint32 - } + _ structs.HostLayout + StartMonoTimeTs uint64 + EndMonoTimeTs uint64 + Bytes uint64 + Packets uint32 + EthProtocol uint16 + Flags uint16 + SrcMac [6]uint8 + DstMac [6]uint8 + IfIndexFirstSeen uint32 Sampling uint32 DirectionFirstSeen uint8 Errno uint8 @@ -138,7 +134,6 @@ type BpfFlowMetricsT struct { ObservedDirection [6]uint8 _ [2]byte ObservedIntf [6]uint32 - _ [4]byte } type BpfFlowRecordT struct { diff --git a/pkg/ebpf/bpf_s390_bpfeb.o b/pkg/ebpf/bpf_s390_bpfeb.o index aa38ced8a..7f5f1ce4a 100644 Binary files a/pkg/ebpf/bpf_s390_bpfeb.o and b/pkg/ebpf/bpf_s390_bpfeb.o differ diff --git a/pkg/ebpf/bpf_x86_bpfel.go b/pkg/ebpf/bpf_x86_bpfel.go index 22c157d7a..9d464053a 100644 --- a/pkg/ebpf/bpf_x86_bpfel.go +++ b/pkg/ebpf/bpf_x86_bpfel.go @@ -116,20 +116,16 @@ type BpfFlowIdT struct { type BpfFlowMetrics BpfFlowMetricsT type BpfFlowMetricsT struct { - _ structs.HostLayout - StartMonoTimeTs uint64 - EndMonoTimeTs uint64 - Bytes uint64 - Packets uint32 - EthProtocol uint16 - Flags uint16 - SrcMac [6]uint8 - DstMac [6]uint8 - IfIndexFirstSeen uint32 - Lock struct { - _ structs.HostLayout - Val uint32 - } + _ structs.HostLayout + StartMonoTimeTs uint64 + EndMonoTimeTs uint64 + Bytes uint64 + Packets uint32 + EthProtocol uint16 + Flags uint16 + SrcMac [6]uint8 + DstMac [6]uint8 + IfIndexFirstSeen uint32 Sampling uint32 DirectionFirstSeen uint8 Errno uint8 @@ -138,7 +134,6 @@ type BpfFlowMetricsT struct { ObservedDirection [6]uint8 _ [2]byte ObservedIntf [6]uint32 - _ [4]byte } type BpfFlowRecordT struct { diff --git a/pkg/ebpf/bpf_x86_bpfel.o b/pkg/ebpf/bpf_x86_bpfel.o index a1fdefe76..33e4d4e75 100644 Binary files a/pkg/ebpf/bpf_x86_bpfel.o and b/pkg/ebpf/bpf_x86_bpfel.o differ diff --git a/pkg/model/record_test.go b/pkg/model/record_test.go index 46ab2eab9..cfa3e0538 100644 --- a/pkg/model/record_test.go +++ b/pkg/model/record_test.go @@ -16,7 +16,7 @@ import ( func TestRecordBinaryEncoding(t *testing.T) { // Makes sure that we read the C *not packed* flow structure according - // to the order defined in bpf/flow.h + // to the order defined in bpf/types.h fr, err := ReadFrom(bytes.NewReader([]byte{ // ID 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x06, 0x07, 0x08, 0x09, // network: u8[16] src_ip @@ -37,7 +37,6 @@ func TestRecordBinaryEncoding(t *testing.T) { 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, // data_link: u8[6] src_mac 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, // data_link: u8[6] dst_mac 0x13, 0x14, 0x15, 0x16, // u32 if_index_first_seen - 0x00, 0x00, 0x00, 0x00, // u32 lock 0x02, 0x00, 0x00, 0x00, // u32 sampling 0x03, // u8 direction_first_seen 0x33, // u8 errno diff --git a/scripts/README_visualize.md b/scripts/README_visualize.md new file mode 100644 index 000000000..14ba97b4f --- /dev/null +++ b/scripts/README_visualize.md @@ -0,0 +1,94 @@ +# eBPF Performance Visualization Script + +## Overview + +`visualize_ebpf_performance.py` creates comprehensive visualizations of eBPF agent performance data from CSV files generated by prow. + +## Usage + +```bash +python3 scripts/visualize_ebpf_performance.py [--output ] [--num-runs ] +``` + +### Arguments + +- `csv_file`: Path to CSV file containing performance data such as `https://gcsweb-ci.apps.ci.l2s4.p1.openshiftapps.com/gcs/test-platform-results/pr-logs/pull/netobserv_netobserv-ebpf-agent/824/pull-ci-netobserv-netobserv-ebpf-agent-main-ebpf-node-density-heavy-25nodes/1985348508604960768/artifacts/ebpf-node-density-heavy-25nodes/openshift-qe-orion/artifacts/data-netobserv-perf-node-density-heavy-AWS-25w.csv` +- `prow_id`: Prow ID of the target run to compare against previous runs such as `1985348508604960768` +- `--output`, `-o`: (Optional) Output PNG file path (default: `perf/ebpf_performance_visualization.png`) +- `--num-runs`, `-n`: (Optional) Number of previous runs to compare against (default: `3`) + +### Examples + +```bash +# Basic usage (compares against last 3 previous runs by default) +python3 scripts/visualize_ebpf_performance.py data.csv 1985348508604960768 + +# Specify custom output file +python3 scripts/visualize_ebpf_performance.py data.csv 1985348508604960768 --output custom_output.png + +# Compare against last 5 previous runs +python3 scripts/visualize_ebpf_performance.py data.csv 1985348508604960768 --num-runs 5 + +# With full paths and custom number of runs +python3 scripts/visualize_ebpf_performance.py /path/to/data.csv 1985348508604960768 -o /path/to/output.png -n 10 +``` + +## Requirements + +- Python 3 +- matplotlib +- numpy + +Install dependencies: +```bash +pip3 install matplotlib numpy +``` + +## Output + +The script generates an 8-panel visualization (5 rows × 2 columns) showing: + +**Row 1:** +1. **Flows Processed Over Time** - Scatter plot with previous average line +2. **Flows Processed Comparison** - Bar chart comparing min/avg/max vs current + +**Row 2:** +3. **CPU Usage Over Time** - CPU usage trends with previous average line +4. **Memory Usage Over Time** - RSS memory usage trends with previous average line + +**Row 3:** +5. **CPU Efficiency** - Flows per minute per core scatter plot with efficiency trend line +6. **Memory Efficiency** - Flows per minute per MB scatter plot with efficiency trend line + +**Row 4:** +7. **Efficiency Comparison** - Percentage change grouped bar chart comparing CPU and Memory efficiency against each of the last N previous runs individually (labeled by prow ID), full width spanning both columns + +**Row 5:** +8. **Summary Statistics** - Comprehensive text panel split into 3 columns: + - **Column 1**: Flows metrics (Flows Processed, Flows Per Minute) + - **Column 2**: Resource usage (CPU Usage, Memory RSS) + - **Column 3**: Efficiency metrics and Dropped Flows information + +## Metrics Visualized + +- **Flows Processed**: Total flows and flows per minute +- **CPU Usage**: Resource consumption (cores) over time +- **Memory Usage**: RSS memory consumption (GB) over time +- **Efficiency Metrics**: + - **CPU Efficiency**: Flows per minute per core (scatter plot + percentage change) + - **Memory Efficiency**: Flows per minute per MB (scatter plot + percentage change) +- **Dropped Flows**: Reliability indicator (zero is ideal) - shown in summary panel only + +## Notes + +- The script compares the target prow ID against the last N previous runs in the CSV (default: 3, configurable with `--num-runs`) +- Efficiency calculations use rate-based metrics (flows per minute) for accurate comparison +- Memory efficiency is calculated per MB (not GB) for more granular analysis +- Efficiency comparison shows percentage change for each individual previous run (labeled by prow ID), allowing you to see variations between runs +- All plots include trend lines and reference averages for easy comparison +- Dropped flows information is displayed in the summary panel (not as a separate plot) +- Summary statistics are organized into 3 columns for better readability +- Output is saved at 300 DPI for high-quality visualization +- By default, output is saved to `perf/` folder (created automatically if it doesn't exist) +- The `perf/` folder is ignored by git (added to `.gitignore`) + diff --git a/scripts/benchmark-images.sh b/scripts/benchmark-images.sh new file mode 100755 index 000000000..85bdf24b5 --- /dev/null +++ b/scripts/benchmark-images.sh @@ -0,0 +1,1403 @@ +#!/usr/bin/env bash +set -euo pipefail + +# Benchmark script to compare BPF performance between two agent images +# +# Usage: +# ./scripts/benchmark-images.sh [options] +# +# Options: +# --duration Duration to run each benchmark (default: 60) +# --output-dir Output directory for results (default: perf/benchmark-) +# --keep-cluster Keep KIND cluster after benchmark (default: cleanup) +# --kubeconfig Use existing kubeconfig file (skips KIND cluster creation) +# --help Show this help message +# +# Example: +# ./scripts/benchmark-images.sh quay.io/user/agent:v1.0.0 quay.io/user/agent:v2.0.0 --duration 300 + +SCRIPT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" +PROJECT_ROOT="$(cd "${SCRIPT_DIR}/.." && pwd)" +OUTPUT_DIR="${OUTPUT_DIR:-}" +DURATION="${DURATION:-300}" # Default 300 seconds (5 minutes) for better averaging +WARMUP_PERIOD="${WARMUP_PERIOD:-180}" # 180 seconds for agent and cluster stabilization +TRAFFIC_WARMUP="${TRAFFIC_WARMUP:-120}" # 120 seconds for traffic to stabilize +CLUSTER_INIT_STABILIZE="${CLUSTER_INIT_STABILIZE:-120}" # 120 seconds for initial cluster stabilization +KEEP_CLUSTER="${KEEP_CLUSTER:-false}" +EXISTING_KUBECONFIG="${EXISTING_KUBECONFIG:-}" +USE_EXISTING_CLUSTER=false +TIMESTAMP=$(date +%Y%m%d-%H%M%S) +CLUSTER_NAME="benchmark-${TIMESTAMP}" +NAMESPACE1="netobserv-privileged-1" +NAMESPACE2="netobserv-privileged-2" +IPERF3_NAMESPACE="iperf3-traffic" + +# Fixed iperf3 parameters for deterministic traffic generation +IPERF3_PARALLEL_STREAMS="${IPERF3_PARALLEL_STREAMS:-8}" # Number of parallel streams +IPERF3_BANDWIDTH="${IPERF3_BANDWIDTH:-100M}" # Bandwidth per stream +IPERF3_WINDOW_SIZE="${IPERF3_WINDOW_SIZE:-128K}" # TCP window size +IPERF3_MSS="${IPERF3_MSS:-1460}" # Maximum segment size (MTU - IP/TCP headers) +IPERF3_TEST_DURATION="${IPERF3_TEST_DURATION:-3600}" # Test duration (infinite, restarted) + +# Colors for output +RED='\033[0;31m' +GREEN='\033[0;32m' +YELLOW='\033[1;33m' +BLUE='\033[0;34m' +NC='\033[0m' # No Color + +log_info() { + echo -e "${BLUE}[INFO]${NC} $*" +} + +log_success() { + echo -e "${GREEN}[SUCCESS]${NC} $*" +} + +log_warn() { + echo -e "${YELLOW}[WARN]${NC} $*" +} + +log_error() { + echo -e "${RED}[ERROR]${NC} $*" +} + +show_help() { + cat < [options] + +Arguments: + image1 First image to benchmark (baseline) + image2 Second image to benchmark (comparison) + +Options: + --duration Duration to run each benchmark in seconds (default: 300) + --output-dir Output directory for results (default: perf/benchmark-) + --keep-cluster Keep KIND cluster after benchmark instead of cleaning up + --kubeconfig Use existing kubeconfig file (skips KIND cluster creation) + --help Show this help message + +Environment Variables (for advanced tuning): + WARMUP_PERIOD Agent warmup period in seconds (default: 180) + TRAFFIC_WARMUP Traffic stabilization period in seconds (default: 120) + CLUSTER_INIT_STABILIZE Initial cluster stabilization in seconds (default: 120) + IPERF3_PARALLEL_STREAMS Number of parallel iperf3 streams (default: 8) + IPERF3_BANDWIDTH Bandwidth per stream (default: 100M) + IPERF3_WINDOW_SIZE TCP window size (default: 128K) + IPERF3_MSS Maximum segment size (default: 1460) + +Note: Benchmarks run SEQUENTIALLY in the same cluster. Each image is tested + independently to ensure accurate statistics collection for all eBPF programs. + +Examples: + # Compare two images with default settings + $0 quay.io/user/agent:v1.0.0 quay.io/user/agent:v2.0.0 + + # Compare with custom duration + $0 image1:tag image2:tag --duration 900 + + # Keep cluster for debugging + $0 image1:tag image2:tag --keep-cluster + + # Use existing cluster (skip KIND creation) + $0 image1:tag image2:tag --kubeconfig ~/.kube/config +EOF +} + +# Parse arguments +IMAGE1="" +IMAGE2="" + +while [[ $# -gt 0 ]]; do + case $1 in + --duration) + DURATION="$2" + shift 2 + ;; + --output-dir) + OUTPUT_DIR="$2" + shift 2 + ;; + --keep-cluster) + KEEP_CLUSTER="true" + shift + ;; + --kubeconfig) + EXISTING_KUBECONFIG="$2" + USE_EXISTING_CLUSTER=true + shift 2 + ;; + --help) + show_help + exit 0 + ;; + -*) + log_error "Unknown option: $1" + show_help + exit 1 + ;; + *) + if [[ -z "$IMAGE1" ]]; then + IMAGE1="$1" + elif [[ -z "$IMAGE2" ]]; then + IMAGE2="$1" + else + log_error "Unexpected argument: $1" + show_help + exit 1 + fi + shift + ;; + esac +done + +if [[ -z "$IMAGE1" ]] || [[ -z "$IMAGE2" ]]; then + log_error "Both image1 and image2 must be provided" + show_help + exit 1 +fi + +# Set output directory +if [[ -z "$OUTPUT_DIR" ]]; then + OUTPUT_DIR="${PROJECT_ROOT}/perf/benchmark-$(date +%Y%m%d-%H%M%S)" +fi +mkdir -p "$OUTPUT_DIR" + +# Initialize kubeconfig file path +if [[ "$USE_EXISTING_CLUSTER" == "true" ]]; then + if [[ ! -f "$EXISTING_KUBECONFIG" ]]; then + log_error "Kubeconfig file not found: $EXISTING_KUBECONFIG" + exit 1 + fi + KUBECONFIG_FILE="$EXISTING_KUBECONFIG" + log_info "Using existing kubeconfig: $KUBECONFIG_FILE" + # Try to get cluster name from kubeconfig context + CLUSTER_NAME=$(kubectl --kubeconfig="$KUBECONFIG_FILE" config view -o jsonpath='{.contexts[0].context.cluster}' 2>/dev/null || echo "existing-cluster") +else + KUBECONFIG_FILE="${TMPDIR:-/tmp}/kubeconfig-${CLUSTER_NAME}" +fi + +log_info "Starting parallel benchmark comparison" +log_info " Image 1 (baseline): $IMAGE1" +log_info " Image 2 (comparison): $IMAGE2" +log_info " Duration: ${DURATION}s (measurement period)" +log_info " Warmup period: ${WARMUP_PERIOD}s (agent stabilization)" +log_info " Traffic warmup: ${TRAFFIC_WARMUP}s (traffic stabilization)" +log_info " Cluster init stabilize: ${CLUSTER_INIT_STABILIZE}s" +log_info " iperf3 parameters: -P ${IPERF3_PARALLEL_STREAMS} -b ${IPERF3_BANDWIDTH} -w ${IPERF3_WINDOW_SIZE} -M ${IPERF3_MSS}" +log_info " Output directory: $OUTPUT_DIR" +log_info " Cluster: $CLUSTER_NAME" +log_info " Namespace 1 (image 1): $NAMESPACE1" +log_info " Namespace 2 (image 2): $NAMESPACE2" +log_info " Running agents SEQUENTIALLY in same cluster for accurate statistics collection" + +# Check prerequisites +if ! command -v kubectl &> /dev/null; then + log_error "kubectl is not installed. Please install it first." + exit 1 +fi + +if [[ "$USE_EXISTING_CLUSTER" == "false" ]]; then + if ! command -v kind &> /dev/null; then + log_error "kind is not installed. Please install it first." + exit 1 + fi +fi + +# Increase file descriptor limits to prevent "too many open files" errors +# This is especially important when running parallel benchmarks +log_info "Checking and increasing file descriptor limits..." +CURRENT_ULIMIT=$(ulimit -n) +TARGET_ULIMIT=100000 +if [[ $CURRENT_ULIMIT -lt $TARGET_ULIMIT ]]; then + if ulimit -n $TARGET_ULIMIT 2>/dev/null; then + log_info " Increased file descriptor limit from $CURRENT_ULIMIT to $TARGET_ULIMIT" + else + log_warn " Could not increase file descriptor limit (current: $CURRENT_ULIMIT)" + log_warn " This may cause 'too many open files' errors. Consider running: ulimit -n 100000" + fi +else + log_info " File descriptor limit is already sufficient: $CURRENT_ULIMIT" +fi + +# Cleanup function +cleanup() { + log_info "Cleaning up benchmark resources..." + + # Clean up traffic generators and their namespace + delete_traffic_generators + + # Clean up packet-counter (in default namespace) + delete_packet_counter + + # Clean up agent namespaces + log_info "Cleaning up agent namespaces..." + kubectl delete namespace "$NAMESPACE1" --ignore-not-found=true --wait=false || true + kubectl delete namespace "$NAMESPACE2" --ignore-not-found=true --wait=false || true + + # Wait a moment for namespace deletion to start + sleep 2 + + # Only clean up cluster if we created it (not using existing) + if [[ "$USE_EXISTING_CLUSTER" == "false" ]]; then + if [[ "$KEEP_CLUSTER" != "true" ]]; then + log_info "Cleaning up KIND cluster..." + kind delete cluster --name "$CLUSTER_NAME" 2>/dev/null || true + rm -f "${KUBECONFIG_FILE:-}" 2>/dev/null || true + else + log_warn "Keeping cluster for inspection" + log_info "To clean up manually:" + log_info " kind delete cluster --name $CLUSTER_NAME" + log_info " kubectl delete namespace $NAMESPACE1 $NAMESPACE2 $IPERF3_NAMESPACE" + log_info "Kubeconfig file: ${KUBECONFIG_FILE:-}" + fi + else + log_info "Using existing cluster - namespaces cleaned up" + fi +} + +trap cleanup EXIT + +# Function to create a KIND cluster with retry logic +create_cluster() { + local cluster_name=$1 + local kubeconfig_file=$2 + local max_retries=3 + local retry_count=0 + + while [[ $retry_count -lt $max_retries ]]; do + if [[ $retry_count -gt 0 ]]; then + log_warn "Retrying cluster creation for $cluster_name (attempt $((retry_count + 1))/$max_retries)..." + # Clean up any partial cluster from previous attempt + kind delete cluster --name "$cluster_name" 2>/dev/null || true + sleep 5 + fi + + log_info "Creating KIND cluster: $cluster_name" + if cat < "$kubeconfig_file" + + # Wait for cluster to be ready + log_info "Waiting for cluster $cluster_name to be ready..." + if KUBECONFIG="$kubeconfig_file" kubectl wait --for=condition=Ready nodes --all --timeout=300s 2>&1; then + # Wait for cluster metrics to stabilize + log_info "Waiting for cluster $cluster_name metrics to stabilize (${CLUSTER_INIT_STABILIZE}s)..." + sleep "$CLUSTER_INIT_STABILIZE" + return 0 + else + log_warn "Cluster $cluster_name created but nodes not ready, will retry..." + kind delete cluster --name "$cluster_name" 2>/dev/null || true + fi + else + log_warn "Failed to create cluster $cluster_name, will retry..." + fi + + retry_count=$((retry_count + 1)) + done + + log_error "Failed to create cluster $cluster_name after $max_retries attempts" + return 1 +} + +# Create cluster or use existing +if [[ "$USE_EXISTING_CLUSTER" == "true" ]]; then + log_info "Using existing cluster from kubeconfig: $KUBECONFIG_FILE" + # Verify cluster is accessible and connected + log_info "Verifying cluster connection..." + if ! kubectl --kubeconfig="$KUBECONFIG_FILE" cluster-info &>/dev/null; then + log_error "Cannot access cluster using provided kubeconfig: $KUBECONFIG_FILE" + log_error "Please ensure the cluster is accessible and you have proper credentials" + exit 1 + fi + # Verify we can actually query the cluster (not just that kubeconfig is valid) + if ! kubectl --kubeconfig="$KUBECONFIG_FILE" get nodes &>/dev/null; then + log_error "Cannot query cluster nodes - cluster may not be accessible or credentials may be invalid" + log_error "Please verify cluster connectivity: kubectl --kubeconfig=\"$KUBECONFIG_FILE\" get nodes" + exit 1 + fi + log_success "Cluster connected and accessible" +else + log_info "Creating KIND cluster..." + if ! create_cluster "$CLUSTER_NAME" "$KUBECONFIG_FILE"; then + log_error "Failed to create cluster ($CLUSTER_NAME)" + exit 1 + fi + log_success "Cluster created and stabilized" +fi + +# Function to run benchmark for a single image +run_benchmark() { + local image=$1 + local label=$2 + local output_file=$3 + local namespace=$4 + + # Set KUBECONFIG for this benchmark + export KUBECONFIG="$KUBECONFIG_FILE" + + log_info "[$namespace] Running benchmark for $label ($image)..." + + # Load image into kind (only for KIND clusters) + if [[ "$USE_EXISTING_CLUSTER" == "false" ]]; then + log_info "[$namespace] Loading image into KIND cluster..." + docker pull "$image" || { log_error "[$namespace] Failed to pull image: $image"; return 1; } + kind load docker-image "$image" --name "$CLUSTER_NAME" + else + log_info "[$namespace] Using existing cluster - assuming image is available in cluster registry" + # For existing clusters, images should be available via registry or already loaded + # User is responsible for ensuring images are accessible + fi + + # Deploy packet-counter collector first + if [[ "$namespace" == "$NAMESPACE1" ]]; then + log_info "[$namespace] Deploying packet-counter collector..." + deploy_packet_counter + else + log_info "[$namespace] Waiting for packet-counter to be ready (deployed by agent 1)..." + sleep 5 + fi + + # Wait a moment for deployment to be created + sleep 2 + + # Wait for packet-counter deployment to be available + log_info "[$namespace] Waiting for packet-counter to be ready..." + if kubectl wait --for=condition=Available deployment/packet-counter -n default --timeout=60s 2>/dev/null; then + log_success "[$namespace] Packet-counter is ready" + else + log_warn "[$namespace] Packet-counter deployment may not be ready, checking pod status..." + # Try waiting for pod as fallback + sleep 5 + kubectl wait --for=condition=Ready pod -l run=packet-counter -n default --timeout=30s 2>/dev/null || true + fi + sleep 3 + + # Delete any existing agent deployment before deploying new one + log_info "[$namespace] Cleaning up any existing agent deployment..." + kubectl delete daemonset netobserv-ebpf-agent -n "$namespace" --ignore-not-found=true + kubectl wait --for=delete pod -l k8s-app=netobserv-ebpf-agent -n "$namespace" --timeout=60s 2>/dev/null || true + sleep 3 # Give time for cleanup + + # Check if we're on OpenShift and set up SCC if needed + local is_openshift=false + if kubectl get crd securitycontextconstraints.security.openshift.io &>/dev/null; then + is_openshift=true + log_info "[$namespace] Detected OpenShift cluster - setting up privileged SCC access..." + + # Create namespace with privileged pod security labels (if not exists) + if ! kubectl get namespace "$namespace" &>/dev/null; then + kubectl create namespace "$namespace" + fi + kubectl label namespace "$namespace" pod-security.kubernetes.io/enforce=privileged pod-security.kubernetes.io/audit=privileged --overwrite + + # Create service account + kubectl create serviceaccount netobserv-ebpf-agent -n "$namespace" --dry-run=client -o yaml | kubectl apply -f - + + # Grant privileged SCC to service account + if ! kubectl get clusterrole netobserv-ebpf-agent-privileged &>/dev/null; then + log_info "[$namespace] Creating ClusterRole for privileged SCC..." + cat </dev/null; then + log_error "[$namespace] Agent pod failed to become ready" + log_info "[$namespace] Checking pod status..." + kubectl get pods -l k8s-app=netobserv-ebpf-agent -n "$namespace" -o wide + log_info "[$namespace] Checking pod events..." + local agent_pod=$(kubectl get pods -l k8s-app=netobserv-ebpf-agent -n "$namespace" -o jsonpath='{.items[0].metadata.name}' 2>/dev/null || echo "") + if [[ -n "$agent_pod" ]]; then + kubectl describe pod "$agent_pod" -n "$namespace" | tail -20 + log_info "[$namespace] Checking pod logs..." + kubectl logs "$agent_pod" -n "$namespace" --tail=50 2>&1 || true + fi + return 1 + fi + + # Get pod name + local pod_name=$(kubectl get pods -n "$namespace" -l k8s-app=netobserv-ebpf-agent -o jsonpath='{.items[0].metadata.name}') + + # Wait a moment for agent to fully load its programs + sleep 3 + + # Wait for agent to stabilize and cluster metrics to settle + log_info "[$namespace] Waiting for agent and cluster metrics to stabilize (${WARMUP_PERIOD}s)..." + sleep "$WARMUP_PERIOD" + + # Deploy traffic generators (only once, shared between agents) + if [[ "$namespace" == "$NAMESPACE1" ]]; then + log_info "[$namespace] Deploying traffic generators..." + deploy_traffic_generators + else + log_info "[$namespace] Traffic generators already deployed by agent 1..." + sleep 5 + fi + + # Wait for traffic generators to be ready + log_info "[$namespace] Waiting for traffic generators to be ready..." + kubectl wait --for=condition=Ready pod -l app=iperf3-server -n "$IPERF3_NAMESPACE" --timeout=60s || true + kubectl wait --for=condition=Ready pod -l app=iperf3-client -n "$IPERF3_NAMESPACE" --timeout=60s || true + + # Wait for iperf3 clients to establish connections and start generating traffic + log_info "[$namespace] Waiting for traffic generators to establish connections and start generating traffic (${TRAFFIC_WARMUP}s)..." + sleep "$TRAFFIC_WARMUP" + + # Get packet-counter pod name for verification + local packet_counter_pod=$(kubectl get pods -n default -l run=packet-counter -o jsonpath='{.items[0].metadata.name}' 2>/dev/null || echo "") + + # Verify traffic is actually flowing before starting stats collection + # Lower threshold to 100 packets/s and allow more time since packet-counter needs 60s window + if ! verify_traffic_flowing "$packet_counter_pod" "$namespace" 100 15 10; then + log_error "Traffic verification failed - iperf3 traffic generators may not be working" + log_error "This will cause inaccurate benchmark results" + log_error "Checking iperf3 client pod status..." + kubectl get pods -l app=iperf3-client -n "$IPERF3_NAMESPACE" -o wide || true + kubectl get pods -l app=iperf3-server -n "$IPERF3_NAMESPACE" -o wide || true + log_error "Checking iperf3 client logs..." + local client_pods=$(kubectl get pods -l app=iperf3-client -n "$IPERF3_NAMESPACE" -o jsonpath='{.items[*].metadata.name}' 2>/dev/null || echo "") + if [[ -n "$client_pods" ]]; then + for client_pod in $client_pods; do + log_error " Logs from $client_pod:" + kubectl logs -n "$IPERF3_NAMESPACE" "$client_pod" --tail=20 2>&1 | head -10 || true + done + fi + log_error "Failing benchmark run to prevent invalid results" + log_error "Please ensure iperf3 traffic generators are working before running benchmarks" + return 1 + fi + + # Additional wait to ensure agent is actively processing traffic before stats collection + # This gives time for flows to accumulate and agent to be in steady state + log_info "[$namespace] Waiting additional 30s to ensure agent is processing traffic steadily..." + sleep 30 + + # Collect kernel stats using collect-kernel-stats (collects all programs) + log_info "[$namespace] Collecting kernel stats for ${DURATION}s using collect-kernel-stats..." + local stats_cmd="/collect-kernel-stats --duration ${DURATION}s" + + if ! kubectl exec -n "$namespace" "$pod_name" -- test -f /collect-kernel-stats 2>/dev/null; then + log_error "[$namespace] collect-kernel-stats binary not found in image: $image" + log_error "[$namespace] The image must include the /collect-kernel-stats binary" + return 1 + fi + + # Collect packet-counter logs in background + # Use timeout to prevent hanging and ensure cleanup + local packet_counter_logs="${output_file%.json}.packet-counter.log" + if [[ -n "$packet_counter_pod" ]]; then + log_info "[$namespace] Starting packet-counter log collection..." + # Use timeout to ensure the log collection doesn't hang indefinitely + # The timeout should be slightly longer than the measurement duration + local timeout_seconds=$((DURATION + 60)) + timeout "$timeout_seconds" kubectl logs -n default "$packet_counter_pod" --tail=0 -f > "$packet_counter_logs" 2>&1 & + local log_pid=$! + fi + + # Run collect-kernel-stats and filter out log messages (lines starting with "time=") + # to get only the JSON output + # Use a temporary file to capture the raw output first + local temp_output=$(mktemp) + + # Add timeout to kubectl exec to prevent hanging (DURATION + buffer for overhead) + local exec_timeout=$((DURATION + 120)) # Add 2 minutes buffer for command overhead + + log_info "[$namespace] Running collect-kernel-stats with timeout of ${exec_timeout}s..." + if timeout "$exec_timeout" kubectl exec -n "$namespace" "$pod_name" -- $stats_cmd > "$temp_output" 2>&1; then + # Stop packet-counter log collection + if [[ -n "${log_pid:-}" ]]; then + kill "$log_pid" 2>/dev/null || true + wait "$log_pid" 2>/dev/null || true + sleep 1 + fi + + # Filter out log messages (lines starting with "time=") to get only JSON + grep -v '^time=' "$temp_output" > "$output_file" + rm -f "$temp_output" + + # Verify we got valid JSON (starts with {) + if [[ ! -s "$output_file" ]] || ! head -n 1 "$output_file" | grep -q '^{'; then + log_error "[$namespace] collect-kernel-stats did not produce valid JSON output" + log_error "[$namespace] Output file contents:" + cat "$output_file" >&2 + return 1 + fi + + # Extract packet-counter stats from logs + if [[ -f "$packet_counter_logs" ]]; then + extract_packet_counter_stats "$packet_counter_logs" "${output_file%.json}.packet-stats.json" + fi + + log_success "[$namespace] Stats collected successfully" + log_success "[$namespace] Benchmark completed for $label" + return 0 + else + local exit_code=$? + # Stop packet-counter log collection on error + if [[ -n "${log_pid:-}" ]]; then + kill "$log_pid" 2>/dev/null || true + wait "$log_pid" 2>/dev/null || true + fi + + # Check if timeout killed the command (exit code 124) + if [[ $exit_code -eq 124 ]]; then + log_error "[$namespace] collect-kernel-stats timed out after ${exec_timeout}s" + log_error "[$namespace] The command exceeded the timeout duration. This may indicate:" + log_error "[$namespace] - The collect-kernel-stats command is hanging" + log_error "[$namespace] - Network issues preventing kubectl exec from completing" + log_error "[$namespace] - The pod may be unresponsive" + else + log_error "[$namespace] collect-kernel-stats failed with exit code $exit_code" + fi + log_error "[$namespace] Raw output:" + cat "$temp_output" >&2 + rm -f "$temp_output" + return 1 + fi +} + +# Collect cluster information (nodes, instance types, etc.) +collect_cluster_info() { + local output_file=$1 + log_info "Collecting cluster information..." + + # Get node information + local nodes_json=$(kubectl get nodes -o json 2>/dev/null || echo "{}") + + if [[ "$nodes_json" == "{}" ]]; then + log_warn "Could not collect cluster information" + return 1 + fi + + # Extract node information using jq if available, otherwise use kubectl + if command -v jq &> /dev/null; then + local num_nodes=$(echo "$nodes_json" | jq '.items | length' 2>/dev/null || echo "0") + # Use jq with proper handling of optional fields using try-catch + local node_info=$(echo "$nodes_json" | jq -r '.items[] | { + name: .metadata.name, + instance_type: (.metadata.labels."node.kubernetes.io/instance-type" // .metadata.labels."beta.kubernetes.io/instance-type" // "unknown"), + zone: (.metadata.labels."topology.kubernetes.io/zone" // .metadata.labels."failure-domain.beta.kubernetes.io/zone" // "unknown"), + arch: .status.nodeInfo.architecture, + os: .status.nodeInfo.operatingSystem, + kernel: .status.nodeInfo.kernelVersion, + kubelet: .status.nodeInfo.kubeletVersion, + cpu: .status.capacity.cpu, + memory: .status.capacity.memory, + pods: .status.capacity.pods + }' 2>/dev/null | jq -s '.' 2>/dev/null || echo "[]") + + # Create cluster info JSON + cat > "$output_file" </dev/null | wc -l) + local node_names=$(kubectl get nodes -o jsonpath='{.items[*].metadata.name}' 2>/dev/null | tr ' ' ',') + local instance_types=$(kubectl get nodes -o jsonpath='{.items[*].metadata.labels.node\.kubernetes\.io/instance-type}' 2>/dev/null || \ + kubectl get nodes -o jsonpath='{.items[*].metadata.labels.beta\.kubernetes\.io/instance-type}' 2>/dev/null || \ + echo "unknown") + + cat > "$output_file" </dev/null; then + log_info "Loading packet-counter image into KIND cluster..." + kind load docker-image "$packet_counter_image" --name "$CLUSTER_NAME" 2>/dev/null || true + else + log_warn "Could not pull packet-counter image, will rely on cluster pull" + fi + else + log_info "Using existing cluster - assuming packet-counter image is available in cluster registry" + fi + + cat </dev/null || echo "") + if [[ -z "$packet_counter_logs" ]] || ! echo "$packet_counter_logs" | grep -q "starting flow collector"; then + log_warn " Packet-counter may not be fully started yet" + fi + + # Check if we're seeing any iperf3 traffic (port 5201) in the flows + local iperf3_flows=$(echo "$packet_counter_logs" | grep ":5201" | wc -l || echo "0") + if [[ $iperf3_flows -gt 0 ]]; then + log_info " Found $iperf3_flows flow entries with port 5201 (iperf3 traffic detected)" + else + log_warn " No iperf3 traffic (port 5201) detected in packet-counter flows yet" + fi + + # Check agent logs to see if it's sending flows (only check current namespace) + if [[ -n "$namespace" ]]; then + local agent_pod=$(kubectl get pods -n "$namespace" -l k8s-app=netobserv-ebpf-agent -o jsonpath='{.items[0].metadata.name}' 2>/dev/null || echo "") + if [[ -n "$agent_pod" ]]; then + log_info " Checking agent status in namespace $namespace..." + local agent_logs=$(kubectl logs -n "$namespace" "$agent_pod" --tail=20 2>/dev/null || echo "") + if echo "$agent_logs" | grep -q "couldn't send flow records"; then + log_warn " Agent appears to have connection issues with packet-counter" + fi + if echo "$agent_logs" | grep -q "connection error\|connection refused\|Unavailable"; then + log_warn " Agent may not be able to connect to packet-counter" + fi + fi + fi + + local attempt=1 + while [[ $attempt -le $max_attempts ]]; do + # Get recent packet-counter logs and look for rate statistics + # Use larger window (100 lines) to catch rate statistics that may appear less frequently + local recent_logs=$(kubectl logs -n default "$packet_counter_pod" --tail=100 2>/dev/null || echo "") + + if [[ -z "$recent_logs" ]]; then + log_warn " Attempt $attempt/$max_attempts: No logs from packet-counter yet, waiting ${wait_seconds}s..." + sleep "$wait_seconds" + ((attempt++)) + continue + fi + + # Try to find rate statistics in either format: + # Format 1: "2025/11/05 10:34:04 === TOTAL RATE: 6783.6 packets/s 335.0 flows/s 196510951.0 bytes/s" + # Format 2: "2025/11/05 10:34:04 6783.6 packets/s. 335.0 flows/s" + local latest_rate=$(echo "$recent_logs" | grep -E "(=== TOTAL RATE:|packets/s\.)" | tail -1) + + if [[ -z "$latest_rate" ]]; then + # Check if we're at least seeing flow entries (even without rate stats) + local flow_entries=$(echo "$recent_logs" | grep -E "^[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+:" | wc -l || echo "0") + local iperf3_in_flows=$(echo "$recent_logs" | grep ":5201" | wc -l || echo "0") + + # Show some diagnostic info on first few attempts + if [[ $attempt -le 3 ]]; then + log_info " Attempt $attempt/$max_attempts: No rate statistics found yet" + log_info " Flow entries in logs: $flow_entries (iperf3 flows: $iperf3_in_flows)" + if [[ $flow_entries -gt 0 ]]; then + log_info " Packet-counter is receiving flows but rate statistics not available yet" + log_info " Packet-counter logs (last 3 lines):" + echo "$recent_logs" | tail -3 | sed 's/^/ /' || true + else + log_info " Packet-counter logs (last 3 lines):" + echo "$recent_logs" | tail -3 | sed 's/^/ /' || true + fi + elif [[ $attempt -eq 5 ]]; then + log_warn " Attempt $attempt/$max_attempts: No rate statistics found yet" + log_warn " Flow entries seen: $flow_entries (iperf3 flows: $iperf3_in_flows)" + if [[ $iperf3_in_flows -gt 0 ]]; then + log_info " iperf3 traffic detected in flows, but rate stats not logging yet" + log_info " This may be normal - packet-counter logs rates every 5 seconds" + fi + else + log_warn " Attempt $attempt/$max_attempts: No rate statistics found yet, waiting ${wait_seconds}s..." + fi + sleep "$wait_seconds" + ((attempt++)) + continue + fi + + # Extract packets/s from the rate line (handle both formats) + local packets_per_sec="0" + if echo "$latest_rate" | grep -q "=== TOTAL RATE:"; then + # Format 1: "=== TOTAL RATE: 6783.6 packets/s ..." + packets_per_sec=$(echo "$latest_rate" | sed -n 's/.*TOTAL RATE: \([0-9.]*\) packets\/s.*/\1/p' || echo "0") + elif echo "$latest_rate" | grep -q "packets/s\."; then + # Format 2: "6783.6 packets/s. ..." + packets_per_sec=$(echo "$latest_rate" | sed -n 's/.* \([0-9.]*\) packets\/s\..*/\1/p' || echo "0") + fi + + # Convert to integer for comparison (using awk to handle floating point) + local packets_int=$(echo "$packets_per_sec" | awk '{printf "%.0f", $1}' 2>/dev/null || echo "0") + local min_int=$(echo "$min_packets_per_sec" | awk '{printf "%.0f", $1}' 2>/dev/null || echo "1000") + + if [[ -z "$packets_per_sec" ]] || [[ "$packets_per_sec" == "0" ]] || [[ "$packets_int" == "0" ]]; then + log_warn " Attempt $attempt/$max_attempts: Traffic rate is 0 packets/s, waiting ${wait_seconds}s..." + if [[ $attempt -eq 5 ]]; then + log_warn " Showing packet-counter logs for diagnostics:" + kubectl logs -n default "$packet_counter_pod" --tail=10 2>/dev/null | sed 's/^/ /' || true + fi + sleep "$wait_seconds" + ((attempt++)) + continue + fi + + if [[ $packets_int -ge $min_int ]]; then + log_success " Traffic verified: ${packets_per_sec} packets/s (threshold: ${min_packets_per_sec} packets/s)" + return 0 + else + # If we have some traffic but below threshold, check if iperf3 is actually flowing + local iperf3_check=$(echo "$recent_logs" | grep ":5201" | head -1 || echo "") + if [[ -n "$iperf3_check" ]]; then + log_warn " Attempt $attempt/$max_attempts: Traffic rate ${packets_per_sec} packets/s is below threshold ${min_packets_per_sec} packets/s" + log_info " iperf3 traffic detected in flows, but rate may be too low" + else + log_warn " Attempt $attempt/$max_attempts: Traffic rate ${packets_per_sec} packets/s is below threshold ${min_packets_per_sec} packets/s" + log_warn " No iperf3 traffic (port 5201) detected in recent flows" + fi + sleep "$wait_seconds" + ((attempt++)) + fi + done + + log_error " Traffic verification failed after $max_attempts attempts" + log_error " Last packet-counter logs:" + kubectl logs -n default "$packet_counter_pod" --tail=10 2>/dev/null | sed 's/^/ /' || true + + # Final check: Only allow proceeding if we have significant traffic AND iperf3 flows + # Check a larger window (500 lines) to account for flow accumulation over time + local final_logs=$(kubectl logs -n default "$packet_counter_pod" --tail=500 2>/dev/null || echo "") + local final_iperf3=$(echo "$final_logs" | grep ":5201" | wc -l || echo "0") + local final_flows=$(echo "$final_logs" | grep -E "^[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+:" | wc -l || echo "0") + + # Require at least 20 iperf3 flows AND 200 total flows to ensure sufficient traffic + # (Reduced thresholds to account for flow accumulation time and log window size) + if [[ $final_iperf3 -ge 20 ]] && [[ $final_flows -ge 200 ]]; then + log_warn " However, detected $final_flows total flows ($final_iperf3 with iperf3) in packet-counter logs" + log_warn " Packet-counter may need more time to accumulate rate statistics (60s window)" + log_warn " Proceeding with benchmark - significant traffic detected" + return 0 # Allow to proceed only with significant traffic + else + log_error " Insufficient traffic detected: $final_flows total flows ($final_iperf3 with iperf3)" + log_error " Required: at least 20 iperf3 flows and 200 total flows (checked last 500 log lines)" + fi + + log_error " This may indicate:" + log_error " 1. iperf3 traffic generators are not working properly" + log_error " 2. Agent is not capturing/sending flows to packet-counter" + log_error " 3. Network connectivity issues between agent and packet-counter" + return 1 +} + +# Extract packet-counter stats from logs +extract_packet_counter_stats() { + local log_file=$1 + local output_file=$2 + + if [[ ! -f "$log_file" ]]; then + return 1 + fi + + # Extract all rate lines and calculate averages + # Format 1: "2025/11/05 10:34:04 === TOTAL RATE: 3.5 packets/s 2.2 flows/s 659.7 bytes/s" + # Format 2: "2025/11/05 10:34:04 3.5 packets/s. 2.2 flows/s" + local rates=$(grep -E "(=== TOTAL RATE:|packets/s\.)" "$log_file" | \ + sed -n -e 's/.*TOTAL RATE: \([0-9.]*\) packets\/s \([0-9.]*\) flows\/s \([0-9.]*\) bytes\/s.*/\1 \2 \3/p' \ + -e 's/.* \([0-9.]*\) packets\/s\. \([0-9.]*\) flows\/s.*/\1 \2 0/p' | \ + grep -v "^$" || echo "") + + if [[ -z "$rates" ]]; then + log_warn " No packet-counter stats found in logs" + return 1 + fi + + # Calculate averages using awk + local avg_packets=$(echo "$rates" | awk '{sum+=$1; count++} END {if(count>0) print sum/count; else print "0"}') + local avg_flows=$(echo "$rates" | awk '{sum+=$2; count++} END {if(count>0) print sum/count; else print "0"}') + local avg_bytes=$(echo "$rates" | awk '{sum+=$3; count++} END {if(count>0) print sum/count; else print "0"}') + + # Get min and max for range + local min_packets=$(echo "$rates" | awk '{if(NR==1 || $1max) max=$1} END {print max+0}') + local min_flows=$(echo "$rates" | awk '{if(NR==1 || $2max) max=$2} END {print max+0}') + + # Create JSON output + cat > "$output_file" </dev/null; then + # Namespace exists, check if it's active (not terminating) + local phase=$(kubectl get namespace "$IPERF3_NAMESPACE" -o jsonpath='{.status.phase}' 2>/dev/null || echo "") + if [[ "$phase" == "Active" ]]; then + log_info "Namespace $IPERF3_NAMESPACE is ready" + break + fi + fi + if [[ $attempt -eq $max_attempts ]]; then + log_error "Failed to create or verify namespace $IPERF3_NAMESPACE after $max_attempts attempts" + return 1 + fi + sleep 1 + ((attempt++)) + done + + cat <.iperf3-server-headless..svc.cluster.local + SERVER_HOST="iperf3-server-\${POD_INDEX}.iperf3-server-headless.${IPERF3_NAMESPACE}.svc.cluster.local" + + # Wait for specific server pod to be ready (check DNS resolution) + echo "Waiting for server: \$SERVER_HOST" + for i in \$(seq 1 30); do + if nslookup \$SERVER_HOST &>/dev/null || ping -c 1 \$SERVER_HOST &>/dev/null; then + echo "Found server at \$SERVER_HOST" + break + fi + sleep 2 + done + + # Generate consistent traffic with fixed parameters + # Use long test duration for steady, continuous traffic + while true; do + echo "Starting iperf3 test to \$SERVER_HOST - client-\${POD_INDEX} to server-\${POD_INDEX}" + # Run continuous test - each client connects to its dedicated server + iperf3 -c \$SERVER_HOST \ + -t 3600 \ + -P "${IPERF3_PARALLEL_STREAMS}" \ + -b "${IPERF3_BANDWIDTH}" \ + -w "${IPERF3_WINDOW_SIZE}" \ + -M "${IPERF3_MSS}" \ + --connect-timeout 10000 \ + --no-delay \ + --get-server-output || { + echo "iperf3 test failed, waiting before retry..." + sleep 10 + } + # Small delay before restarting + sleep 1 + done + resources: + requests: + memory: "32Mi" + cpu: "50m" + limits: + memory: "128Mi" + cpu: "1000m" + affinity: + podAffinity: + preferredDuringSchedulingIgnoredDuringExecution: + - weight: 100 + podAffinityTerm: + labelSelector: + matchLabels: + app: iperf3-server + topologyKey: kubernetes.io/hostname + tolerations: + - key: node-role.kubernetes.io/control-plane + operator: Exists + effect: NoSchedule +EOF +} + +# Delete traffic generators +delete_traffic_generators() { + log_info "Cleaning up traffic generators..." + kubectl delete statefulset iperf3-server iperf3-client -n "$IPERF3_NAMESPACE" --ignore-not-found=true || true + kubectl delete service iperf3-server iperf3-server-headless iperf3-client-headless -n "$IPERF3_NAMESPACE" --ignore-not-found=true || true + kubectl delete namespace "$IPERF3_NAMESPACE" --ignore-not-found=true || true + sleep 3 +} + +# Delete existing agent before deploying new one +delete_agent() { + local namespace=$1 + log_info "[$namespace] Deleting existing agent deployment..." + kubectl delete daemonset netobserv-ebpf-agent -n "$namespace" --ignore-not-found=true + kubectl wait --for=delete pod -l k8s-app=netobserv-ebpf-agent -n "$namespace" --timeout=60s || true + sleep 5 +} + +# Run benchmarks sequentially +OUTPUT_FILE1="${OUTPUT_DIR}/kernel-stats-image1.json" +OUTPUT_FILE2="${OUTPUT_DIR}/kernel-stats-image2.json" +CLUSTER_INFO_FILE="${OUTPUT_DIR}/cluster-info.json" + +# Ensure output directory exists before writing log files +mkdir -p "$OUTPUT_DIR" + +# Collect cluster information once (shared between both benchmarks) +log_info "Collecting cluster information..." +if ! collect_cluster_info "$CLUSTER_INFO_FILE"; then + log_warn "Failed to collect cluster information, continuing without it..." +fi + +# Run benchmarks sequentially in the same cluster +log_info "Starting sequential benchmarks in single cluster..." + +# Run benchmark 1 +log_info "=== Running benchmark for Image 1 (baseline) ===" +if ! run_benchmark "$IMAGE1" "Image 1 - baseline" "$OUTPUT_FILE1" "$NAMESPACE1" > "${OUTPUT_DIR}/benchmark-image1.log" 2>&1; then + log_error "Benchmark for image 1 failed" + log_info "Check logs: ${OUTPUT_DIR}/benchmark-image1.log" + exit 1 +fi +log_success "Benchmark for Image 1 completed successfully" + +# Clean up agent 1 before starting agent 2 +log_info "Cleaning up Image 1 agent before starting Image 2..." +delete_agent "$NAMESPACE1" +log_info "Waiting 30s for agent 1 to fully unload eBPF programs and cleanup..." +sleep 30 # Give time for eBPF programs to be unloaded and cleanup to complete + +# Run benchmark 2 +log_info "=== Running benchmark for Image 2 (comparison) ===" +if ! run_benchmark "$IMAGE2" "Image 2 - comparison" "$OUTPUT_FILE2" "$NAMESPACE2" > "${OUTPUT_DIR}/benchmark-image2.log" 2>&1; then + log_error "Benchmark for image 2 failed" + log_info "Check logs: ${OUTPUT_DIR}/benchmark-image2.log" + exit 1 +fi +log_success "Benchmark for Image 2 completed successfully" + +log_success "Both benchmarks completed successfully" + +# Generate comparison report +log_info "Generating comparison report..." +REPORT_GENERATED=false + +if [[ ! -f "$OUTPUT_FILE1" ]]; then + log_error "Cannot generate comparison report: Image 1 stats file missing: $OUTPUT_FILE1" +elif [[ ! -f "$OUTPUT_FILE2" ]]; then + log_error "Cannot generate comparison report: Image 2 stats file missing: $OUTPUT_FILE2" +elif [[ ! -f "${SCRIPT_DIR}/compare-kernel-stats.py" ]]; then + log_error "Comparison script not found: ${SCRIPT_DIR}/compare-kernel-stats.py" +else + # Build packet stats arguments if files exist + packet_stats_args="" + if [[ -f "${OUTPUT_FILE1%.json}.packet-stats.json" ]]; then + packet_stats_args="${packet_stats_args} --baseline-packet-stats ${OUTPUT_FILE1%.json}.packet-stats.json" + log_info " Including packet stats for image 1" + fi + if [[ -f "${OUTPUT_FILE2%.json}.packet-stats.json" ]]; then + packet_stats_args="${packet_stats_args} --comparison-packet-stats ${OUTPUT_FILE2%.json}.packet-stats.json" + log_info " Including packet stats for image 2" + fi + + # Use default labels if IMAGE1/IMAGE2 are not set (shouldn't happen, but safety check) + baseline_label="${IMAGE1:-Image1}" + comparison_label="${IMAGE2:-Image2}" + if [[ "$baseline_label" != "Image1" ]] && [[ "$comparison_label" != "Image2" ]]; then + baseline_label="$(basename "$IMAGE1")" + comparison_label="$(basename "$IMAGE2")" + fi + + log_info " Generating report with baseline: $baseline_label, comparison: $comparison_label" + + if python3 "${SCRIPT_DIR}/compare-kernel-stats.py" \ + --baseline "$OUTPUT_FILE1" \ + --comparison "$OUTPUT_FILE2" \ + --output "${OUTPUT_DIR}/comparison-report.png" \ + --baseline-label "$baseline_label" \ + --comparison-label "$comparison_label" \ + $packet_stats_args 2>&1; then + log_success "Comparison report generated successfully" + REPORT_GENERATED=true + else + local exit_code=$? + log_error "Comparison report generation failed with exit code: $exit_code" + log_error "Attempting to run comparison script manually to see error:" + python3 "${SCRIPT_DIR}/compare-kernel-stats.py" \ + --baseline "$OUTPUT_FILE1" \ + --comparison "$OUTPUT_FILE2" \ + --output "${OUTPUT_DIR}/comparison-report.png" \ + --baseline-label "$baseline_label" \ + --comparison-label "$comparison_label" \ + $packet_stats_args 2>&1 || true + fi +fi + +log_success "Benchmark completed!" +log_info "Results saved to: $OUTPUT_DIR" +log_info " - Image 1 stats: $OUTPUT_FILE1" +log_info " - Image 2 stats: $OUTPUT_FILE2" +if [[ -f "${OUTPUT_FILE1%.json}.packet-stats.json" ]]; then + log_info " - Image 1 packet stats: ${OUTPUT_FILE1%.json}.packet-stats.json" +fi +if [[ -f "${OUTPUT_FILE2%.json}.packet-stats.json" ]]; then + log_info " - Image 2 packet stats: ${OUTPUT_FILE2%.json}.packet-stats.json" +fi +if [[ -f "${OUTPUT_DIR}/comparison-report.png" ]]; then + log_info " - Comparison report: ${OUTPUT_DIR}/comparison-report.png" +else + log_warn " - Comparison report: NOT GENERATED (check errors above)" +fi + diff --git a/scripts/compare-kernel-stats.py b/scripts/compare-kernel-stats.py new file mode 100755 index 000000000..ee76c97ce --- /dev/null +++ b/scripts/compare-kernel-stats.py @@ -0,0 +1,1011 @@ +#!/usr/bin/env python3 +""" +Compare kernel stats from two benchmark runs and generate matplotlib visualizations. + +Usage: + python3 compare-kernel-stats.py --baseline --comparison --output +""" + +import json +import argparse +import sys +from pathlib import Path +from datetime import datetime +import matplotlib +matplotlib.use('Agg') # Use non-interactive backend +import matplotlib.pyplot as plt +import numpy as np + + +def load_stats(filepath): + """Load kernel stats from JSON file.""" + try: + with open(filepath, 'r') as f: + return json.load(f) + except FileNotFoundError: + print(f"Error: File not found: {filepath}", file=sys.stderr) + sys.exit(1) + except json.JSONDecodeError as e: + print(f"Error: Invalid JSON in {filepath}: {e}", file=sys.stderr) + sys.exit(1) + + +def calculate_percentage_change(old_val, new_val): + """Calculate percentage change between two values.""" + if old_val == 0: + if new_val == 0: + return 0.0 + return float('inf') if new_val > 0 else float('-inf') + return ((new_val - old_val) / old_val) * 100 + + +def generate_matplotlib_report(baseline_stats, comparison_stats, baseline_label, comparison_label, output_file, + baseline_packet_stats=None, comparison_packet_stats=None, cluster_info=None): + """Generate matplotlib visualization report as separate graph files.""" + + baseline_programs = baseline_stats.get('program_stats', {}) + comparison_programs = comparison_stats.get('program_stats', {}) + + # Collect all program names (only those with activity) + all_programs = [] + for prog_name in sorted(set(baseline_programs.keys()) | set(comparison_programs.keys())): + baseline_prog = baseline_programs.get(prog_name, {}) + comparison_prog = comparison_programs.get(prog_name, {}) + # Only include programs with activity in at least one run + if (baseline_prog.get('events_per_sec', 0) > 0 or + comparison_prog.get('events_per_sec', 0) > 0 or + baseline_prog.get('estimated_cpu_percent', 0) > 0 or + comparison_prog.get('estimated_cpu_percent', 0) > 0): + all_programs.append(prog_name or '(unnamed)') + + # Calculate totals + baseline_total_cpu = baseline_stats.get('total_estimated_cpu_percent', 0) + comparison_total_cpu = comparison_stats.get('total_estimated_cpu_percent', 0) + cpu_change = calculate_percentage_change(baseline_total_cpu, comparison_total_cpu) + + baseline_total_events = baseline_stats.get('total_events_per_sec', 0) + comparison_total_events = comparison_stats.get('total_events_per_sec', 0) + events_change = calculate_percentage_change(baseline_total_events, comparison_total_events) + + baseline_total_runtime = baseline_stats.get('total_runtime_ns', 0) + comparison_total_runtime = comparison_stats.get('total_runtime_ns', 0) + + # Calculate average runtime per event (total runtime / total run count) + baseline_run_count = baseline_stats.get('total_run_count', 0) + comparison_run_count = comparison_stats.get('total_run_count', 0) + + baseline_avg_runtime = baseline_total_runtime / baseline_run_count if baseline_run_count > 0 else 0 + comparison_avg_runtime = comparison_total_runtime / comparison_run_count if comparison_run_count > 0 else 0 + avg_runtime_change = calculate_percentage_change(baseline_avg_runtime, comparison_avg_runtime) + + # Collection duration is stored in nanoseconds - handle both old and new field names + duration_ns = baseline_stats.get('collection_duration_ns') or baseline_stats.get('collection_duration_sec', 0) + duration_sec = duration_ns / 1e9 + + # Load and process packet stats if available + baseline_packets_per_sec = 0 + baseline_flows_per_sec = 0 + baseline_bytes_per_sec = 0 + comparison_packets_per_sec = 0 + comparison_flows_per_sec = 0 + comparison_bytes_per_sec = 0 + + if baseline_packet_stats: + baseline_packets_per_sec = baseline_packet_stats.get('avg_packets_per_sec', 0) + baseline_flows_per_sec = baseline_packet_stats.get('avg_flows_per_sec', 0) + baseline_bytes_per_sec = baseline_packet_stats.get('avg_bytes_per_sec', 0) + + if comparison_packet_stats: + comparison_packets_per_sec = comparison_packet_stats.get('avg_packets_per_sec', 0) + comparison_flows_per_sec = comparison_packet_stats.get('avg_flows_per_sec', 0) + comparison_bytes_per_sec = comparison_packet_stats.get('avg_bytes_per_sec', 0) + + # Calculate efficiency metrics + # CPU efficiency: packets per second per CPU percent + baseline_cpu_efficiency_packets = baseline_packets_per_sec / baseline_total_cpu if baseline_total_cpu > 0 else 0 + comparison_cpu_efficiency_packets = comparison_packets_per_sec / comparison_total_cpu if comparison_total_cpu > 0 else 0 + cpu_efficiency_packets_change = calculate_percentage_change(baseline_cpu_efficiency_packets, comparison_cpu_efficiency_packets) + + # CPU efficiency: flows per second per CPU percent + baseline_cpu_efficiency_flows = baseline_flows_per_sec / baseline_total_cpu if baseline_total_cpu > 0 else 0 + comparison_cpu_efficiency_flows = comparison_flows_per_sec / comparison_total_cpu if comparison_total_cpu > 0 else 0 + cpu_efficiency_flows_change = calculate_percentage_change(baseline_cpu_efficiency_flows, comparison_cpu_efficiency_flows) + + # CPU efficiency: bytes per second per CPU percent + baseline_cpu_efficiency_bytes = baseline_bytes_per_sec / baseline_total_cpu if baseline_total_cpu > 0 else 0 + comparison_cpu_efficiency_bytes = comparison_bytes_per_sec / comparison_total_cpu if comparison_total_cpu > 0 else 0 + cpu_efficiency_bytes_change = calculate_percentage_change(baseline_cpu_efficiency_bytes, comparison_cpu_efficiency_bytes) + + # Events efficiency: packets per event + baseline_events_efficiency_packets = baseline_packets_per_sec / baseline_total_events if baseline_total_events > 0 else 0 + comparison_events_efficiency_packets = comparison_packets_per_sec / comparison_total_events if comparison_total_events > 0 else 0 + events_efficiency_packets_change = calculate_percentage_change(baseline_events_efficiency_packets, comparison_events_efficiency_packets) + + # Packets/Flows/Bytes changes + packets_change = calculate_percentage_change(baseline_packets_per_sec, comparison_packets_per_sec) + flows_change = calculate_percentage_change(baseline_flows_per_sec, comparison_flows_per_sec) + bytes_change = calculate_percentage_change(baseline_bytes_per_sec, comparison_bytes_per_sec) + + # Calculate program metadata totals + baseline_total_memlock = 0 + baseline_total_jited_size = 0 + baseline_total_verified_insns = 0 + baseline_total_maps = 0 + comparison_total_memlock = 0 + comparison_total_jited_size = 0 + comparison_total_verified_insns = 0 + comparison_total_maps = 0 + + for prog_name in all_programs: + baseline_prog = baseline_programs.get(prog_name, {}) + comparison_prog = comparison_programs.get(prog_name, {}) + baseline_total_memlock += baseline_prog.get('memlock_bytes', 0) + baseline_total_jited_size += baseline_prog.get('jited_size_bytes', 0) + baseline_total_verified_insns += baseline_prog.get('verified_instructions', 0) + baseline_total_maps += baseline_prog.get('num_maps', 0) + comparison_total_memlock += comparison_prog.get('memlock_bytes', 0) + comparison_total_jited_size += comparison_prog.get('jited_size_bytes', 0) + comparison_total_verified_insns += comparison_prog.get('verified_instructions', 0) + comparison_total_maps += comparison_prog.get('num_maps', 0) + + memlock_change = calculate_percentage_change(baseline_total_memlock, comparison_total_memlock) + jited_size_change = calculate_percentage_change(baseline_total_jited_size, comparison_total_jited_size) + verified_insns_change = calculate_percentage_change(baseline_total_verified_insns, comparison_total_verified_insns) + maps_change = calculate_percentage_change(baseline_total_maps, comparison_total_maps) + + # Extract base path from output_file to create separate graph files + output_path = Path(output_file) + base_dir = output_path.parent + base_name = output_path.stem # filename without extension + + # Set style + plt.style.use('default') + + generated_files = [] + + # Helper function to save a figure + def save_figure(fig, filename, title): + """Save a figure to a file with title.""" + fig.suptitle(title, fontsize=14, fontweight='bold') + fig.text(0.99, 0.01, f'Generated: {datetime.now().strftime("%Y-%m-%d %H:%M:%S")}', + ha='right', va='bottom', fontsize=8, style='italic') + filepath = base_dir / filename + fig.savefig(filepath, dpi=150, bbox_inches='tight') + plt.close(fig) + generated_files.append(filepath) + print(f"Graph generated: {filepath}") + + # Store plot data for combined figure + plot_data = [] + + # 1. CPU Usage Comparison (by program) + fig1 = plt.figure(figsize=(14, 8)) + ax1 = fig1.add_subplot(1, 1, 1) + baseline_cpus = [] + comparison_cpus = [] + program_labels = [] + + for prog_name in all_programs: + baseline_prog = baseline_programs.get(prog_name, {}) + comparison_prog = comparison_programs.get(prog_name, {}) + baseline_cpu = baseline_prog.get('estimated_cpu_percent', 0) + comparison_cpu = comparison_prog.get('estimated_cpu_percent', 0) + + if baseline_cpu > 0 or comparison_cpu > 0: + baseline_cpus.append(baseline_cpu) + comparison_cpus.append(comparison_cpu) + # Truncate long program names + label = prog_name[:20] + '...' if len(prog_name) > 20 else prog_name + program_labels.append(label) + + if baseline_cpus: + x = np.arange(len(program_labels)) + width = 0.35 + + bars1 = ax1.bar(x - width/2, baseline_cpus, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax1.bar(x + width/2, comparison_cpus, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + ax1.set_xlabel('Program', fontweight='bold') + ax1.set_ylabel('CPU Usage (%)', fontweight='bold') + ax1.set_title('CPU Usage by Program', fontweight='bold', fontsize=12) + ax1.set_xticks(x) + ax1.set_xticklabels(program_labels, rotation=45, ha='right', fontsize=8) + ax1.legend() + ax1.grid(True, alpha=0.3, axis='y') + ax1.set_yscale('log') + + save_figure(fig1, f'{base_name}-cpu-usage.png', + f'CPU Usage by Program\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + else: + plt.close(fig1) + + # 2. Events/sec Comparison (by program) + fig2 = plt.figure(figsize=(14, 8)) + ax2 = fig2.add_subplot(1, 1, 1) + baseline_events = [] + comparison_events = [] + program_labels2 = [] + + for prog_name in all_programs: + baseline_prog = baseline_programs.get(prog_name, {}) + comparison_prog = comparison_programs.get(prog_name, {}) + baseline_evt = baseline_prog.get('events_per_sec', 0) + comparison_evt = comparison_prog.get('events_per_sec', 0) + + if baseline_evt > 0 or comparison_evt > 0: + baseline_events.append(baseline_evt) + comparison_events.append(comparison_evt) + label = prog_name[:20] + '...' if len(prog_name) > 20 else prog_name + program_labels2.append(label) + + if baseline_events: + x = np.arange(len(program_labels2)) + width = 0.35 + + bars1 = ax2.bar(x - width/2, baseline_events, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax2.bar(x + width/2, comparison_events, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + ax2.set_xlabel('Program', fontweight='bold') + ax2.set_ylabel('Events/sec', fontweight='bold') + ax2.set_title('Events/sec by Program', fontweight='bold', fontsize=12) + ax2.set_xticks(x) + ax2.set_xticklabels(program_labels2, rotation=45, ha='right', fontsize=8) + ax2.legend() + ax2.grid(True, alpha=0.3, axis='y') + ax2.set_yscale('log') + + save_figure(fig2, f'{base_name}-events-per-sec.png', + f'Events/sec by Program\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + else: + plt.close(fig2) + + # 3a. CPU Usage Comparison + fig3a = plt.figure(figsize=(8, 6)) + ax3a = fig3a.add_subplot(1, 1, 1) + metrics_cpu = ['CPU %'] + baseline_values_cpu = [baseline_total_cpu] + comparison_values_cpu = [comparison_total_cpu] + + x = np.arange(len(metrics_cpu)) + width = 0.35 + + bars1 = ax3a.bar(x - width/2, baseline_values_cpu, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax3a.bar(x + width/2, comparison_values_cpu, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + # Add percentage change label + max_val = max(baseline_total_cpu, comparison_total_cpu) if max(baseline_total_cpu, comparison_total_cpu) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(cpu_change) < 1 else ('green' if cpu_change < 0 else 'red') + ax3a.text(0, y_pos, f'{cpu_change:+.2f}%', ha='center', va='bottom', + fontsize=10, fontweight='bold', color=color) + + ax3a.set_ylabel('CPU Usage (%)', fontweight='bold') + ax3a.set_title('Total CPU Usage Comparison', fontweight='bold', fontsize=12) + ax3a.set_xticks(x) + ax3a.set_xticklabels(metrics_cpu) + ax3a.legend() + ax3a.grid(True, alpha=0.3, axis='y') + + save_figure(fig3a, f'{base_name}-cpu-total.png', + f'Total CPU Usage Comparison\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + + # 3b. Events/sec Comparison + fig3b = plt.figure(figsize=(8, 6)) + ax3b = fig3b.add_subplot(1, 1, 1) + metrics_events = ['Events/s'] + baseline_values_events = [baseline_total_events] + comparison_values_events = [comparison_total_events] + + x = np.arange(len(metrics_events)) + width = 0.35 + + bars1 = ax3b.bar(x - width/2, baseline_values_events, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax3b.bar(x + width/2, comparison_values_events, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + # Add percentage change label + max_val = max(baseline_total_events, comparison_total_events) if max(baseline_total_events, comparison_total_events) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(events_change) < 1 else ('green' if events_change > 0 else 'red') + ax3b.text(0, y_pos, f'{events_change:+.2f}%', ha='center', va='bottom', + fontsize=10, fontweight='bold', color=color) + + ax3b.set_ylabel('Events/sec', fontweight='bold') + ax3b.set_title('Total Events/sec Comparison', fontweight='bold', fontsize=12) + ax3b.set_xticks(x) + ax3b.set_xticklabels(metrics_events) + ax3b.legend() + ax3b.grid(True, alpha=0.3, axis='y') + + save_figure(fig3b, f'{base_name}-events-total.png', + f'Total Events/sec Comparison\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + + # 3c. Average Runtime Comparison + fig3c = plt.figure(figsize=(8, 6)) + ax3c = fig3c.add_subplot(1, 1, 1) + metrics_runtime = ['Avg Runtime (μs)'] + baseline_values_runtime = [baseline_avg_runtime / 1e3] # Convert to microseconds + comparison_values_runtime = [comparison_avg_runtime / 1e3] + + x = np.arange(len(metrics_runtime)) + width = 0.35 + + bars1 = ax3c.bar(x - width/2, baseline_values_runtime, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax3c.bar(x + width/2, comparison_values_runtime, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + # Add percentage change label + max_val = max(baseline_values_runtime[0], comparison_values_runtime[0]) if max(baseline_values_runtime[0], comparison_values_runtime[0]) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(avg_runtime_change) < 1 else ('green' if avg_runtime_change < 0 else 'red') + ax3c.text(0, y_pos, f'{avg_runtime_change:+.2f}%', ha='center', va='bottom', + fontsize=10, fontweight='bold', color=color) + + ax3c.set_ylabel('Average Runtime (μs)', fontweight='bold') + ax3c.set_title('Average Runtime per Event Comparison', fontweight='bold', fontsize=12) + ax3c.set_xticks(x) + ax3c.set_xticklabels(metrics_runtime) + ax3c.legend() + ax3c.grid(True, alpha=0.3, axis='y') + + save_figure(fig3c, f'{base_name}-avg-runtime-total.png', + f'Average Runtime per Event Comparison\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + + # 4. Average Runtime Comparison (by program) + fig4 = plt.figure(figsize=(14, 8)) + ax4 = fig4.add_subplot(1, 1, 1) + baseline_runtimes = [] + comparison_runtimes = [] + program_labels3 = [] + + for prog_name in all_programs: + baseline_prog = baseline_programs.get(prog_name, {}) + comparison_prog = comparison_programs.get(prog_name, {}) + baseline_rt = baseline_prog.get('avg_runtime_ns', 0) / 1e3 # Convert to microseconds + comparison_rt = comparison_prog.get('avg_runtime_ns', 0) / 1e3 + + if baseline_rt > 0 or comparison_rt > 0: + baseline_runtimes.append(baseline_rt) + comparison_runtimes.append(comparison_rt) + label = prog_name[:20] + '...' if len(prog_name) > 20 else prog_name + program_labels3.append(label) + + if baseline_runtimes: + x = np.arange(len(program_labels3)) + width = 0.35 + + bars1 = ax4.bar(x - width/2, baseline_runtimes, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax4.bar(x + width/2, comparison_runtimes, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + ax4.set_xlabel('Program', fontweight='bold') + ax4.set_ylabel('Avg Runtime (μs)', fontweight='bold') + ax4.set_title('Average Runtime by Program', fontweight='bold', fontsize=12) + ax4.set_xticks(x) + ax4.set_xticklabels(program_labels3, rotation=45, ha='right', fontsize=8) + ax4.legend() + ax4.grid(True, alpha=0.3, axis='y') + ax4.set_yscale('log') + + save_figure(fig4, f'{base_name}-avg-runtime.png', + f'Average Runtime by Program\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + else: + plt.close(fig4) + + # 5. Packet Stats Comparison (if available) + if baseline_packet_stats or comparison_packet_stats: + fig5 = plt.figure(figsize=(10, 6)) + ax5 = fig5.add_subplot(1, 1, 1) + packet_metrics = ['Packets/s', 'Flows/s', 'Bytes/s (MB)'] + baseline_packet_values = [ + baseline_packets_per_sec, + baseline_flows_per_sec, + baseline_bytes_per_sec / 1e6 # Convert to MB/s + ] + comparison_packet_values = [ + comparison_packets_per_sec, + comparison_flows_per_sec, + comparison_bytes_per_sec / 1e6 # Convert to MB/s + ] + + x = np.arange(len(packet_metrics)) + width = 0.35 + + bars1 = ax5.bar(x - width/2, baseline_packet_values, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax5.bar(x + width/2, comparison_packet_values, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + # Add percentage change labels + packet_changes = [packets_change, flows_change, bytes_change] + for i, (base, comp, change) in enumerate(zip(baseline_packet_values, comparison_packet_values, packet_changes)): + max_val = max(base, comp) if max(base, comp) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(change) < 1 else ('green' if change > 0 else 'red') + ax5.text(i, y_pos, f'{change:+.1f}%', ha='center', va='bottom', + fontsize=9, fontweight='bold', color=color) + + ax5.set_xlabel('Metric', fontweight='bold') + ax5.set_ylabel('Rate', fontweight='bold') + ax5.set_title('Packet/Flow Statistics', fontweight='bold', fontsize=12) + ax5.set_xticks(x) + ax5.set_xticklabels(packet_metrics) + ax5.legend() + ax5.grid(True, alpha=0.3, axis='y') + ax5.set_yscale('log') + + save_figure(fig5, f'{base_name}-packet-stats.png', + f'Packet/Flow Statistics\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + + # 6. Efficiency Metrics Comparison (if packet stats available) + if baseline_packet_stats or comparison_packet_stats: + fig6 = plt.figure(figsize=(10, 6)) + ax6 = fig6.add_subplot(1, 1, 1) + efficiency_metrics = ['Pkts/s per CPU%', 'Flows/s per CPU%', 'MB/s per CPU%'] + baseline_efficiency_values = [ + baseline_cpu_efficiency_packets, + baseline_cpu_efficiency_flows, + baseline_cpu_efficiency_bytes / 1e6 # Convert to MB/s per CPU% + ] + comparison_efficiency_values = [ + comparison_cpu_efficiency_packets, + comparison_cpu_efficiency_flows, + comparison_cpu_efficiency_bytes / 1e6 # Convert to MB/s per CPU% + ] + + x = np.arange(len(efficiency_metrics)) + width = 0.35 + + bars1 = ax6.bar(x - width/2, baseline_efficiency_values, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax6.bar(x + width/2, comparison_efficiency_values, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + # Add percentage change labels + efficiency_changes = [cpu_efficiency_packets_change, cpu_efficiency_flows_change, cpu_efficiency_bytes_change] + for i, (base, comp, change) in enumerate(zip(baseline_efficiency_values, comparison_efficiency_values, efficiency_changes)): + max_val = max(base, comp) if max(base, comp) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(change) < 1 else ('green' if change > 0 else 'red') + ax6.text(i, y_pos, f'{change:+.1f}%', ha='center', va='bottom', + fontsize=9, fontweight='bold', color=color) + + ax6.set_xlabel('Metric', fontweight='bold') + ax6.set_ylabel('Efficiency', fontweight='bold') + ax6.set_title('CPU Efficiency Metrics', fontweight='bold', fontsize=12) + ax6.set_xticks(x) + ax6.set_xticklabels(efficiency_metrics, rotation=15, ha='right') + ax6.legend() + ax6.grid(True, alpha=0.3, axis='y') + ax6.set_yscale('log') + + save_figure(fig6, f'{base_name}-cpu-efficiency.png', + f'CPU Efficiency Metrics\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + + # 7. Program Metadata Comparison (Memory, JIT Size, Instructions, Maps) + fig7 = plt.figure(figsize=(10, 6)) + ax7 = fig7.add_subplot(1, 1, 1) + metadata_metrics = [] + baseline_metadata_values = [] + comparison_metadata_values = [] + metadata_changes = [] + + if baseline_total_memlock > 0 or comparison_total_memlock > 0: + metadata_metrics.append('Memory (MB)') + baseline_metadata_values.append(baseline_total_memlock / 1e6) + comparison_metadata_values.append(comparison_total_memlock / 1e6) + metadata_changes.append(memlock_change) + + if baseline_total_jited_size > 0 or comparison_total_jited_size > 0: + metadata_metrics.append('JIT Size (KB)') + baseline_metadata_values.append(baseline_total_jited_size / 1e3) + comparison_metadata_values.append(comparison_total_jited_size / 1e3) + metadata_changes.append(jited_size_change) + + if baseline_total_verified_insns > 0 or comparison_total_verified_insns > 0: + metadata_metrics.append('Instructions') + baseline_metadata_values.append(baseline_total_verified_insns) + comparison_metadata_values.append(comparison_total_verified_insns) + metadata_changes.append(verified_insns_change) + + if baseline_total_maps > 0 or comparison_total_maps > 0: + metadata_metrics.append('Maps') + baseline_metadata_values.append(baseline_total_maps) + comparison_metadata_values.append(comparison_total_maps) + metadata_changes.append(maps_change) + + if metadata_metrics: + x = np.arange(len(metadata_metrics)) + width = 0.35 + + bars1 = ax7.bar(x - width/2, baseline_metadata_values, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax7.bar(x + width/2, comparison_metadata_values, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + # Add percentage change labels + for i, (base, comp, change) in enumerate(zip(baseline_metadata_values, comparison_metadata_values, metadata_changes)): + max_val = max(base, comp) if max(base, comp) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(change) < 1 else ('green' if change < 0 else 'red') + ax7.text(i, y_pos, f'{change:+.1f}%', ha='center', va='bottom', + fontsize=9, fontweight='bold', color=color) + + ax7.set_xlabel('Metric', fontweight='bold') + ax7.set_ylabel('Value', fontweight='bold') + ax7.set_title('Program Metadata Comparison', fontweight='bold', fontsize=12) + ax7.set_xticks(x) + ax7.set_xticklabels(metadata_metrics, rotation=15, ha='right') + ax7.legend() + ax7.grid(True, alpha=0.3, axis='y') + ax7.set_yscale('log') + + save_figure(fig7, f'{base_name}-program-metadata.png', + f'Program Metadata Comparison\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + else: + plt.close(fig7) + + # 8. Program Metadata by Program (Memory footprint) + fig8 = plt.figure(figsize=(14, 8)) + ax8 = fig8.add_subplot(1, 1, 1) + baseline_memlocks = [] + comparison_memlocks = [] + program_labels4 = [] + + for prog_name in all_programs: + baseline_prog = baseline_programs.get(prog_name, {}) + comparison_prog = comparison_programs.get(prog_name, {}) + baseline_mem = baseline_prog.get('memlock_bytes', 0) / 1e6 # Convert to MB + comparison_mem = comparison_prog.get('memlock_bytes', 0) / 1e6 + + if baseline_mem > 0 or comparison_mem > 0: + baseline_memlocks.append(baseline_mem) + comparison_memlocks.append(comparison_mem) + label = prog_name[:20] + '...' if len(prog_name) > 20 else prog_name + program_labels4.append(label) + + if baseline_memlocks: + x = np.arange(len(program_labels4)) + width = 0.35 + + bars1 = ax8.bar(x - width/2, baseline_memlocks, width, label=baseline_label, color='#3498db', alpha=0.8) + bars2 = ax8.bar(x + width/2, comparison_memlocks, width, label=comparison_label, color='#e74c3c', alpha=0.8) + + ax8.set_xlabel('Program', fontweight='bold') + ax8.set_ylabel('Memory (MB)', fontweight='bold') + ax8.set_title('Memory Footprint by Program', fontweight='bold', fontsize=12) + ax8.set_xticks(x) + ax8.set_xticklabels(program_labels4, rotation=45, ha='right', fontsize=8) + ax8.legend() + ax8.grid(True, alpha=0.3, axis='y') + ax8.set_yscale('log') + + save_figure(fig8, f'{base_name}-memory-footprint.png', + f'Memory Footprint by Program\nBaseline: {baseline_label} vs Comparison: {comparison_label}') + else: + plt.close(fig8) + + # Generate summary text file + summary_lines = [] + + # Add cluster information if available + if cluster_info: + summary_lines.append("Cluster Information:") + summary_lines.append(f" Cluster Name: {cluster_info.get('cluster_name', 'unknown')}") + summary_lines.append(f" Cluster Type: {cluster_info.get('cluster_type', 'unknown')}") + summary_lines.append(f" Number of Nodes: {cluster_info.get('num_nodes', 'unknown')}") + + if 'nodes' in cluster_info and isinstance(cluster_info['nodes'], list): + # Detailed node information + instance_types = {} + for node in cluster_info['nodes']: + inst_type = node.get('instance_type', 'unknown') + instance_types[inst_type] = instance_types.get(inst_type, 0) + 1 + + if instance_types: + summary_lines.append(" Instance Types:") + for inst_type, count in sorted(instance_types.items()): + summary_lines.append(f" - {inst_type}: {count} node(s)") + + # Show node details if there are few nodes + if len(cluster_info['nodes']) <= 5: + summary_lines.append(" Node Details:") + for node in cluster_info['nodes']: + summary_lines.append(f" - {node.get('name', 'unknown')}: {node.get('instance_type', 'unknown')} " + f"({node.get('cpu', '?')} CPU, {node.get('arch', '?')} arch)") + elif 'instance_types' in cluster_info: + # Fallback format + summary_lines.append(f" Instance Types: {cluster_info.get('instance_types', 'unknown')}") + + summary_lines.append("") # Empty line separator + + summary_lines.extend([ + f"Collection Duration: {duration_sec:.0f}s", + f"Total CPU: Baseline {baseline_total_cpu:.4f}% | Comparison {comparison_total_cpu:.4f}% | Change: {cpu_change:+.2f}%", + f"Total Events/sec: Baseline {baseline_total_events:.2f}/s | Comparison {comparison_total_events:.2f}/s | Change: {events_change:+.2f}%", + f"Avg Runtime/Event: Baseline {baseline_avg_runtime/1e3:.2f}μs | Comparison {comparison_avg_runtime/1e3:.2f}μs | Change: {avg_runtime_change:+.2f}%" + ]) + + if baseline_packet_stats or comparison_packet_stats: + summary_lines.append(f"Packets/sec: Baseline {baseline_packets_per_sec:.2f}/s | Comparison {comparison_packets_per_sec:.2f}/s | Change: {packets_change:+.2f}%") + summary_lines.append(f"Flows/sec: Baseline {baseline_flows_per_sec:.2f}/s | Comparison {comparison_flows_per_sec:.2f}/s | Change: {flows_change:+.2f}%") + summary_lines.append(f"Bytes/sec: Baseline {baseline_bytes_per_sec/1e6:.2f}MB/s | Comparison {comparison_bytes_per_sec/1e6:.2f}MB/s | Change: {bytes_change:+.2f}%") + summary_lines.append(f"CPU Efficiency (pkts/s per %): Baseline {baseline_cpu_efficiency_packets:.2f} | Comparison {comparison_cpu_efficiency_packets:.2f} | Change: {cpu_efficiency_packets_change:+.2f}%") + summary_lines.append(f"CPU Efficiency (flows/s per %): Baseline {baseline_cpu_efficiency_flows:.2f} | Comparison {comparison_cpu_efficiency_flows:.2f} | Change: {cpu_efficiency_flows_change:+.2f}%") + + # Add program metadata to summary + if baseline_total_memlock > 0 or comparison_total_memlock > 0: + summary_lines.append(f"Total Memory (MB): Baseline {baseline_total_memlock/1e6:.2f}MB | Comparison {comparison_total_memlock/1e6:.2f}MB | Change: {memlock_change:+.2f}%") + if baseline_total_jited_size > 0 or comparison_total_jited_size > 0: + summary_lines.append(f"Total JIT Size (KB): Baseline {baseline_total_jited_size/1e3:.2f}KB | Comparison {comparison_total_jited_size/1e3:.2f}KB | Change: {jited_size_change:+.2f}%") + if baseline_total_verified_insns > 0 or comparison_total_verified_insns > 0: + summary_lines.append(f"Total Instructions: Baseline {baseline_total_verified_insns} | Comparison {comparison_total_verified_insns} | Change: {verified_insns_change:+.2f}%") + if baseline_total_maps > 0 or comparison_total_maps > 0: + summary_lines.append(f"Total Maps: Baseline {baseline_total_maps} | Comparison {comparison_total_maps} | Change: {maps_change:+.2f}%") + + # Create combined figure with all graphs + # Calculate number of graphs to include + num_graphs = 5 # cpu-usage, events-per-sec, cpu-total, events-total, avg-runtime-total + if baseline_runtimes: + num_graphs += 1 # avg-runtime by program + if baseline_packet_stats or comparison_packet_stats: + num_graphs += 2 # packet-stats, cpu-efficiency + if metadata_metrics: + num_graphs += 1 # program-metadata + if baseline_memlocks: + num_graphs += 1 # memory-footprint + + # Create grid layout (3 columns) + cols = 3 + rows = (num_graphs + cols - 1) // cols + + combined_fig = plt.figure(figsize=(20, 6 * rows)) + combined_fig.suptitle(f'eBPF Performance Comparison\nBaseline: {baseline_label} vs Comparison: {comparison_label}', + fontsize=16, fontweight='bold', y=0.995) + + subplot_idx = 1 + + # 1. CPU Usage by Program + if baseline_cpus: + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + x = np.arange(len(program_labels)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_cpus, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_cpus, width, label=comparison_label, color='#e74c3c', alpha=0.8) + ax_combined.set_xlabel('Program', fontweight='bold', fontsize=9) + ax_combined.set_ylabel('CPU Usage (%)', fontweight='bold', fontsize=9) + ax_combined.set_title('CPU Usage by Program', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(program_labels, rotation=45, ha='right', fontsize=7) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + ax_combined.set_yscale('log') + subplot_idx += 1 + + # 2. Events/sec by Program + if baseline_events: + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + x = np.arange(len(program_labels2)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_events, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_events, width, label=comparison_label, color='#e74c3c', alpha=0.8) + ax_combined.set_xlabel('Program', fontweight='bold', fontsize=9) + ax_combined.set_ylabel('Events/sec', fontweight='bold', fontsize=9) + ax_combined.set_title('Events/sec by Program', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(program_labels2, rotation=45, ha='right', fontsize=7) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + ax_combined.set_yscale('log') + subplot_idx += 1 + + # 3. Total CPU Usage + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + metrics_cpu = ['CPU %'] + baseline_values_cpu = [baseline_total_cpu] + comparison_values_cpu = [comparison_total_cpu] + x = np.arange(len(metrics_cpu)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_values_cpu, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_values_cpu, width, label=comparison_label, color='#e74c3c', alpha=0.8) + max_val = max(baseline_total_cpu, comparison_total_cpu) if max(baseline_total_cpu, comparison_total_cpu) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(cpu_change) < 1 else ('green' if cpu_change < 0 else 'red') + ax_combined.text(0, y_pos, f'{cpu_change:+.2f}%', ha='center', va='bottom', fontsize=9, fontweight='bold', color=color) + ax_combined.set_ylabel('CPU Usage (%)', fontweight='bold', fontsize=9) + ax_combined.set_title('Total CPU Usage', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(metrics_cpu, fontsize=9) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + subplot_idx += 1 + + # 4. Total Events/sec + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + metrics_events = ['Events/s'] + baseline_values_events = [baseline_total_events] + comparison_values_events = [comparison_total_events] + x = np.arange(len(metrics_events)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_values_events, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_values_events, width, label=comparison_label, color='#e74c3c', alpha=0.8) + max_val = max(baseline_total_events, comparison_total_events) if max(baseline_total_events, comparison_total_events) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(events_change) < 1 else ('green' if events_change > 0 else 'red') + ax_combined.text(0, y_pos, f'{events_change:+.2f}%', ha='center', va='bottom', fontsize=9, fontweight='bold', color=color) + ax_combined.set_ylabel('Events/sec', fontweight='bold', fontsize=9) + ax_combined.set_title('Total Events/sec', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(metrics_events, fontsize=9) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + subplot_idx += 1 + + # 5. Average Runtime per Event + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + metrics_runtime = ['Avg Runtime (μs)'] + baseline_values_runtime = [baseline_avg_runtime / 1e3] + comparison_values_runtime = [comparison_avg_runtime / 1e3] + x = np.arange(len(metrics_runtime)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_values_runtime, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_values_runtime, width, label=comparison_label, color='#e74c3c', alpha=0.8) + max_val = max(baseline_values_runtime[0], comparison_values_runtime[0]) if max(baseline_values_runtime[0], comparison_values_runtime[0]) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(avg_runtime_change) < 1 else ('green' if avg_runtime_change < 0 else 'red') + ax_combined.text(0, y_pos, f'{avg_runtime_change:+.2f}%', ha='center', va='bottom', fontsize=9, fontweight='bold', color=color) + ax_combined.set_ylabel('Average Runtime (μs)', fontweight='bold', fontsize=9) + ax_combined.set_title('Avg Runtime per Event', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(metrics_runtime, fontsize=9) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + subplot_idx += 1 + + # 6. Average Runtime by Program (if available) + if baseline_runtimes: + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + x = np.arange(len(program_labels3)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_runtimes, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_runtimes, width, label=comparison_label, color='#e74c3c', alpha=0.8) + ax_combined.set_xlabel('Program', fontweight='bold', fontsize=9) + ax_combined.set_ylabel('Avg Runtime (μs)', fontweight='bold', fontsize=9) + ax_combined.set_title('Avg Runtime by Program', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(program_labels3, rotation=45, ha='right', fontsize=7) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + ax_combined.set_yscale('log') + subplot_idx += 1 + + # 7. Packet Stats (if available) + if baseline_packet_stats or comparison_packet_stats: + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + packet_metrics = ['Packets/s', 'Flows/s', 'Bytes/s (MB)'] + baseline_packet_values = [ + baseline_packets_per_sec, + baseline_flows_per_sec, + baseline_bytes_per_sec / 1e6 + ] + comparison_packet_values = [ + comparison_packets_per_sec, + comparison_flows_per_sec, + comparison_bytes_per_sec / 1e6 + ] + x = np.arange(len(packet_metrics)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_packet_values, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_packet_values, width, label=comparison_label, color='#e74c3c', alpha=0.8) + packet_changes = [packets_change, flows_change, bytes_change] + for i, (base, comp, change) in enumerate(zip(baseline_packet_values, comparison_packet_values, packet_changes)): + max_val = max(base, comp) if max(base, comp) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(change) < 1 else ('green' if change > 0 else 'red') + ax_combined.text(i, y_pos, f'{change:+.1f}%', ha='center', va='bottom', fontsize=8, fontweight='bold', color=color) + ax_combined.set_xlabel('Metric', fontweight='bold', fontsize=9) + ax_combined.set_ylabel('Rate', fontweight='bold', fontsize=9) + ax_combined.set_title('Packet/Flow Statistics', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(packet_metrics, fontsize=8) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + ax_combined.set_yscale('log') + subplot_idx += 1 + + # 8. CPU Efficiency (if available) + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + efficiency_metrics = ['Pkts/s per CPU%', 'Flows/s per CPU%', 'MB/s per CPU%'] + baseline_efficiency_values = [ + baseline_cpu_efficiency_packets, + baseline_cpu_efficiency_flows, + baseline_cpu_efficiency_bytes / 1e6 + ] + comparison_efficiency_values = [ + comparison_cpu_efficiency_packets, + comparison_cpu_efficiency_flows, + comparison_cpu_efficiency_bytes / 1e6 + ] + x = np.arange(len(efficiency_metrics)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_efficiency_values, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_efficiency_values, width, label=comparison_label, color='#e74c3c', alpha=0.8) + efficiency_changes = [cpu_efficiency_packets_change, cpu_efficiency_flows_change, cpu_efficiency_bytes_change] + for i, (base, comp, change) in enumerate(zip(baseline_efficiency_values, comparison_efficiency_values, efficiency_changes)): + max_val = max(base, comp) if max(base, comp) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(change) < 1 else ('green' if change > 0 else 'red') + ax_combined.text(i, y_pos, f'{change:+.1f}%', ha='center', va='bottom', fontsize=8, fontweight='bold', color=color) + ax_combined.set_xlabel('Metric', fontweight='bold', fontsize=9) + ax_combined.set_ylabel('Efficiency', fontweight='bold', fontsize=9) + ax_combined.set_title('CPU Efficiency Metrics', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(efficiency_metrics, rotation=15, ha='right', fontsize=8) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + ax_combined.set_yscale('log') + subplot_idx += 1 + + # 9. Program Metadata (if available) + if metadata_metrics: + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + x = np.arange(len(metadata_metrics)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_metadata_values, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_metadata_values, width, label=comparison_label, color='#e74c3c', alpha=0.8) + for i, (base, comp, change) in enumerate(zip(baseline_metadata_values, comparison_metadata_values, metadata_changes)): + max_val = max(base, comp) if max(base, comp) > 0 else 1 + y_pos = max_val * 1.1 + color = 'green' if abs(change) < 1 else ('green' if change < 0 else 'red') + ax_combined.text(i, y_pos, f'{change:+.1f}%', ha='center', va='bottom', fontsize=8, fontweight='bold', color=color) + ax_combined.set_xlabel('Metric', fontweight='bold', fontsize=9) + ax_combined.set_ylabel('Value', fontweight='bold', fontsize=9) + ax_combined.set_title('Program Metadata', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(metadata_metrics, rotation=15, ha='right', fontsize=8) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + ax_combined.set_yscale('log') + subplot_idx += 1 + + # 10. Memory Footprint by Program (if available) + if baseline_memlocks: + ax_combined = combined_fig.add_subplot(rows, cols, subplot_idx) + x = np.arange(len(program_labels4)) + width = 0.35 + ax_combined.bar(x - width/2, baseline_memlocks, width, label=baseline_label, color='#3498db', alpha=0.8) + ax_combined.bar(x + width/2, comparison_memlocks, width, label=comparison_label, color='#e74c3c', alpha=0.8) + ax_combined.set_xlabel('Program', fontweight='bold', fontsize=9) + ax_combined.set_ylabel('Memory (MB)', fontweight='bold', fontsize=9) + ax_combined.set_title('Memory Footprint by Program', fontweight='bold', fontsize=10) + ax_combined.set_xticks(x) + ax_combined.set_xticklabels(program_labels4, rotation=45, ha='right', fontsize=7) + ax_combined.legend(fontsize=8) + ax_combined.grid(True, alpha=0.3, axis='y') + ax_combined.set_yscale('log') + subplot_idx += 1 + + # Add timestamp to combined figure + combined_fig.text(0.99, 0.01, f'Generated: {datetime.now().strftime("%Y-%m-%d %H:%M:%S")}', + ha='right', va='bottom', fontsize=8, style='italic') + + # Adjust layout and save combined figure + plt.tight_layout(rect=[0, 0.03, 1, 0.97]) + combined_fig.savefig(output_file, dpi=150, bbox_inches='tight') + plt.close(combined_fig) + print(f"Combined comparison report generated: {output_file}") + generated_files.append(Path(output_file)) + + summary_text = "\n".join(summary_lines) + summary_file = base_dir / f'{base_name}-summary.txt' + with open(summary_file, 'w') as f: + f.write(f"eBPF Performance Comparison Summary\n") + f.write(f"Baseline: {baseline_label} vs Comparison: {comparison_label}\n") + f.write(f"{'='*60}\n\n") + f.write(summary_text) + f.write(f"\n\n{'='*60}\n") + f.write(f"Generated Graphs:\n") + f.write(f"{'='*60}\n\n") + f.write(f"Combined Report:\n") + f.write(f" - {output_path.name}\n\n") + f.write(f"Individual Graphs:\n") + + # List all generated graph files + graph_files = [ + (f'{base_name}-cpu-usage.png', 'CPU Usage by Program'), + (f'{base_name}-events-per-sec.png', 'Events/sec by Program'), + (f'{base_name}-cpu-total.png', 'Total CPU Usage Comparison'), + (f'{base_name}-events-total.png', 'Total Events/sec Comparison'), + (f'{base_name}-avg-runtime-total.png', 'Average Runtime per Event Comparison'), + (f'{base_name}-avg-runtime.png', 'Average Runtime by Program'), + ] + + if baseline_packet_stats or comparison_packet_stats: + graph_files.extend([ + (f'{base_name}-packet-stats.png', 'Packet/Flow Statistics'), + (f'{base_name}-cpu-efficiency.png', 'CPU Efficiency Metrics'), + ]) + + if metadata_metrics: + graph_files.append((f'{base_name}-program-metadata.png', 'Program Metadata Comparison')) + + if baseline_memlocks: + graph_files.append((f'{base_name}-memory-footprint.png', 'Memory Footprint by Program')) + + for filename, description in graph_files: + filepath = base_dir / filename + if filepath.exists(): + f.write(f" - {filename} ({description})\n") + + f.write(f"\nGenerated: {datetime.now().strftime('%Y-%m-%d %H:%M:%S')}\n") + print(f"Summary generated: {summary_file}") + generated_files.append(summary_file) + + print(f"\nAll comparison graphs generated ({len(generated_files)} files)") + print(f"Base output directory: {base_dir}") + + +def main(): + parser = argparse.ArgumentParser( + description='Compare kernel stats from two benchmark runs and generate matplotlib visualizations', + formatter_class=argparse.RawDescriptionHelpFormatter + ) + parser.add_argument('--baseline', required=True, help='Baseline kernel stats JSON file') + parser.add_argument('--comparison', required=True, help='Comparison kernel stats JSON file') + parser.add_argument('--output', required=True, help='Output PNG image file') + parser.add_argument('--baseline-label', default='Baseline', help='Label for baseline image') + parser.add_argument('--comparison-label', default='Comparison', help='Label for comparison image') + parser.add_argument('--baseline-packet-stats', help='Baseline packet stats JSON file (optional)') + parser.add_argument('--comparison-packet-stats', help='Comparison packet stats JSON file (optional)') + parser.add_argument('--cluster-info', help='Cluster information JSON file (optional, auto-detected if in same directory as baseline)') + + args = parser.parse_args() + + # Auto-detect cluster-info.json in the same directory as baseline file if not provided + cluster_info = None + if args.cluster_info: + cluster_info_path = Path(args.cluster_info) + else: + # Try to find cluster-info.json in the same directory as baseline file + baseline_path = Path(args.baseline) + cluster_info_path = baseline_path.parent / 'cluster-info.json' + + if cluster_info_path.exists(): + try: + cluster_info = load_stats(str(cluster_info_path)) + print(f"Loaded cluster information from: {cluster_info_path}") + except (FileNotFoundError, json.JSONDecodeError) as e: + print(f"Warning: Could not load cluster info from {cluster_info_path}: {e}", file=sys.stderr) + cluster_info = None + + baseline_stats = load_stats(args.baseline) + comparison_stats = load_stats(args.comparison) + + baseline_packet_stats = None + comparison_packet_stats = None + + if args.baseline_packet_stats: + if Path(args.baseline_packet_stats).exists(): + try: + baseline_packet_stats = load_stats(args.baseline_packet_stats) + except (FileNotFoundError, json.JSONDecodeError) as e: + print(f"Warning: Could not load baseline packet stats from {args.baseline_packet_stats}: {e}", file=sys.stderr) + baseline_packet_stats = None + else: + print(f"Warning: Baseline packet stats file not found: {args.baseline_packet_stats}", file=sys.stderr) + + if args.comparison_packet_stats: + if Path(args.comparison_packet_stats).exists(): + try: + comparison_packet_stats = load_stats(args.comparison_packet_stats) + except (FileNotFoundError, json.JSONDecodeError) as e: + print(f"Warning: Could not load comparison packet stats from {args.comparison_packet_stats}: {e}", file=sys.stderr) + comparison_packet_stats = None + else: + print(f"Warning: Comparison packet stats file not found: {args.comparison_packet_stats}", file=sys.stderr) + + generate_matplotlib_report( + baseline_stats, + comparison_stats, + args.baseline_label, + args.comparison_label, + args.output, + baseline_packet_stats, + comparison_packet_stats, + cluster_info + ) + + +if __name__ == '__main__': + main() diff --git a/scripts/visualize_ebpf_performance.py b/scripts/visualize_ebpf_performance.py new file mode 100755 index 000000000..fed734e6c --- /dev/null +++ b/scripts/visualize_ebpf_performance.py @@ -0,0 +1,486 @@ +#!/usr/bin/env python3 +""" +Visualization script for eBPF agent performance data from CSV files. + +Usage: + python3 visualize_ebpf_performance.py [--output ] [--num-runs ] + +Example: + python3 visualize_ebpf_performance.py data.csv 1985348508604960768 + python3 visualize_ebpf_performance.py data.csv 1985348508604960768 --output perf.png + python3 visualize_ebpf_performance.py data.csv 1985348508604960768 --num-runs 5 +""" +import csv +import argparse +import os +import sys +import matplotlib +matplotlib.use('Agg') # Use non-interactive backend +import matplotlib.pyplot as plt +import matplotlib.patches as mpatches +import numpy as np + +def extract_prow_id(build_url): + """Extract prow ID from build URL.""" + if build_url and '/' in build_url: + parts = build_url.split('/') + return parts[-1] if parts else None + return None + +def parse_csv_row(row): + """Parse a CSV row and extract relevant fields.""" + return { + 'uuid': row[1], + 'nFlowsProcessedTotals_max': float(row[2]) if row[2] else 0, + 'nFlowsProcessedPerMinuteTotals_max': float(row[3]) if row[3] else 0, + 'ebpfFlowsDroppedRatio_avg': float(row[8]) if row[8] else 0, + 'cpuEBPFTotals_avg': float(row[13]) if row[13] else 0, + 'rssEBPFTotals_avg': float(row[18]) if len(row) > 18 and row[18] else 0, # RSS memory in bytes + 'timestamp': row[34], + 'buildUrl': row[36] + } + +def visualize_csv_data(csv_file, target_prow_id, output_file=None, num_runs=3): + """Create visualizations from CSV data. + + Args: + csv_file: Path to CSV file with performance data + target_prow_id: Prow ID of the target run to compare + output_file: Optional output PNG file path + num_runs: Number of previous runs to compare against (default: 3) + """ + + # Read CSV data + with open(csv_file, 'r') as f: + reader = csv.reader(f) + header = next(reader) + + rows = [] + target_row = None + target_index = None + + for i, row in enumerate(reader): + if len(row) < 37: + continue + + prow_id = extract_prow_id(row[36]) + parsed_row = parse_csv_row(row) + parsed_row['prow_id'] = prow_id + parsed_row['index'] = i + + rows.append(parsed_row) + + if prow_id == target_prow_id: + target_row = parsed_row + target_index = i + + if not target_row: + print(f"Error: Prow ID {target_prow_id} not found in CSV file") + sys.exit(1) + + # Get all previous runs + all_previous_runs = rows[:target_index] + + # Use last N runs for comparison (or all if less than N available) + previous_runs = all_previous_runs[-num_runs:] if len(all_previous_runs) >= num_runs else all_previous_runs + + print(f"Comparing target run to last {len(previous_runs)} previous runs (out of {len(all_previous_runs)} total)") + + all_runs = rows[:target_index + 1] # Include target + + # Create figure with subplots + fig = plt.figure(figsize=(16, 22)) + gs = fig.add_gridspec(6, 2, hspace=0.3, wspace=0.3) + + # Color scheme + color_prev = '#3498db' # Blue + color_current = '#e74c3c' # Red + color_avg = '#2ecc71' # Green + + # 1. Flows Processed Over Time + ax1 = fig.add_subplot(gs[0, 0]) + indices = [r['index'] for r in all_runs] + flows = [r['nFlowsProcessedTotals_max'] / 1e6 for r in all_runs] + colors = [color_current if r['prow_id'] == target_prow_id else color_prev for r in all_runs] + + ax1.scatter(indices, flows, c=colors, alpha=0.6, s=50) + if previous_runs: + avg_flows = sum(r['nFlowsProcessedTotals_max'] for r in previous_runs) / len(previous_runs) / 1e6 + ax1.axhline(y=avg_flows, color=color_avg, linestyle='--', linewidth=2, label=f'Last {len(previous_runs)} Avg: {avg_flows:.2f}M') + ax1.axhline(y=target_row['nFlowsProcessedTotals_max'] / 1e6, color=color_current, linestyle='--', linewidth=2, alpha=0.5, label=f'Current: {target_row["nFlowsProcessedTotals_max"]/1e6:.2f}M') + ax1.set_xlabel('Run Index', fontsize=11, fontweight='bold') + ax1.set_ylabel('Flows Processed (Millions)', fontsize=11, fontweight='bold') + ax1.set_title('Flows Processed Over Time', fontsize=13, fontweight='bold') + ax1.grid(True, alpha=0.3) + ax1.legend(fontsize=9) + + # 2. Comparison Bar Chart + ax2 = fig.add_subplot(gs[0, 1]) + if previous_runs: + prev_avg_flows = sum(r['nFlowsProcessedTotals_max'] for r in previous_runs) / len(previous_runs) / 1e6 + prev_min_flows = min(r['nFlowsProcessedTotals_max'] for r in previous_runs) / 1e6 + prev_max_flows = max(r['nFlowsProcessedTotals_max'] for r in previous_runs) / 1e6 + curr_flows = target_row['nFlowsProcessedTotals_max'] / 1e6 + + categories = [f'Last {len(previous_runs)}\nMin', f'Last {len(previous_runs)}\nAvg', f'Last {len(previous_runs)}\nMax', 'Current\n(Updated)'] + values = [prev_min_flows, prev_avg_flows, prev_max_flows, curr_flows] + colors_bar = [color_prev, color_prev, color_prev, color_current] + bars = ax2.bar(categories, values, color=colors_bar, alpha=0.7, edgecolor='black', linewidth=1.5) + + # Add value labels on bars + for bar, val in zip(bars, values): + height = bar.get_height() + ax2.text(bar.get_x() + bar.get_width()/2., height, + f'{val:.2f}M', + ha='center', va='bottom', fontweight='bold', fontsize=10) + + ax2.set_ylabel('Flows Processed (Millions)', fontsize=11, fontweight='bold') + ax2.set_title('Flows Processed Comparison', fontsize=13, fontweight='bold') + ax2.grid(True, alpha=0.3, axis='y') + + # 3. CPU Usage Comparison + ax3 = fig.add_subplot(gs[1, 0]) + cpu_values = [r['cpuEBPFTotals_avg'] for r in all_runs] + colors_cpu = [color_current if r['prow_id'] == target_prow_id else color_prev for r in all_runs] + + ax3.scatter(indices, cpu_values, c=colors_cpu, alpha=0.6, s=50) + if previous_runs: + avg_cpu = sum(r['cpuEBPFTotals_avg'] for r in previous_runs) / len(previous_runs) + ax3.axhline(y=avg_cpu, color=color_avg, linestyle='--', linewidth=2, label=f'Last {len(previous_runs)} Avg: {avg_cpu:.3f}') + ax3.axhline(y=target_row['cpuEBPFTotals_avg'], color=color_current, linestyle='--', linewidth=2, alpha=0.5, label=f'Current: {target_row["cpuEBPFTotals_avg"]:.3f}') + ax3.set_xlabel('Run Index', fontsize=11, fontweight='bold') + ax3.set_ylabel('CPU Usage (cores)', fontsize=11, fontweight='bold') + ax3.set_title('eBPF CPU Usage Over Time', fontsize=13, fontweight='bold') + ax3.grid(True, alpha=0.3) + ax3.legend(fontsize=9) + + # 4. Memory Usage Over Time + ax4 = fig.add_subplot(gs[1, 1]) + mem_values = [r['rssEBPFTotals_avg'] / 1e9 for r in all_runs] # Convert to GB + colors_mem = [color_current if r['prow_id'] == target_prow_id else color_prev for r in all_runs] + + ax4.scatter(indices, mem_values, c=colors_mem, alpha=0.6, s=50) + if previous_runs: + avg_mem = sum(r['rssEBPFTotals_avg'] for r in previous_runs) / len(previous_runs) / 1e9 + ax4.axhline(y=avg_mem, color=color_avg, linestyle='--', linewidth=2, label=f'Last {len(previous_runs)} Avg: {avg_mem:.2f} GB') + ax4.axhline(y=target_row['rssEBPFTotals_avg'] / 1e9, color=color_current, linestyle='--', linewidth=2, alpha=0.5, label=f'Current: {target_row["rssEBPFTotals_avg"]/1e9:.2f} GB') + ax4.set_xlabel('Run Index', fontsize=11, fontweight='bold') + ax4.set_ylabel('Memory Usage RSS (GB)', fontsize=11, fontweight='bold') + ax4.set_title('eBPF Memory Usage Over Time', fontsize=13, fontweight='bold') + ax4.grid(True, alpha=0.3) + ax4.legend(fontsize=9) + + # 5. CPU vs Flows Per Minute Efficiency + ax5 = fig.add_subplot(gs[2, 0]) + if previous_runs: + prev_cpu = [r['cpuEBPFTotals_avg'] for r in previous_runs] + prev_flows_per_min = [r['nFlowsProcessedPerMinuteTotals_max'] / 1e6 for r in previous_runs] + curr_cpu = target_row['cpuEBPFTotals_avg'] + curr_flows_per_min = target_row['nFlowsProcessedPerMinuteTotals_max'] / 1e6 + + ax5.scatter(prev_flows_per_min, prev_cpu, c=color_prev, alpha=0.6, s=50, label='Previous Runs') + ax5.scatter(curr_flows_per_min, curr_cpu, c=color_current, s=200, marker='*', + edgecolor='black', linewidth=2, label='Current (Updated)', zorder=5) + + # Add efficiency line (flows per minute per core) + if prev_flows_per_min: + x_range = np.linspace(min(prev_flows_per_min), max(max(prev_flows_per_min), curr_flows_per_min), 100) + # Calculate average efficiency using flows per minute + efficiencies = [f / c for f, c in zip(prev_flows_per_min, prev_cpu)] + avg_efficiency = sum(efficiencies) / len(efficiencies) + y_line = x_range / avg_efficiency + ax5.plot(x_range, y_line, '--', color='gray', alpha=0.5, linewidth=1, + label=f'Last {len(previous_runs)} Avg Efficiency: {avg_efficiency:.1f}M flows/min/core') + + ax5.set_xlabel('Flows Per Minute (Millions)', fontsize=11, fontweight='bold') + ax5.set_ylabel('CPU Usage (cores)', fontsize=11, fontweight='bold') + ax5.set_title('CPU Efficiency (Flows/Min per Core)', fontsize=13, fontweight='bold') + ax5.grid(True, alpha=0.3) + ax5.legend(fontsize=9) + + # 6. Memory vs Flows Per Minute Efficiency + ax6 = fig.add_subplot(gs[2, 1]) + if previous_runs: + prev_mem = [r['rssEBPFTotals_avg'] / 1e6 for r in previous_runs] # Convert to MB + prev_flows_per_min = [r['nFlowsProcessedPerMinuteTotals_max'] / 1e6 for r in previous_runs] + curr_mem = target_row['rssEBPFTotals_avg'] / 1e6 # Convert to MB + curr_flows_per_min = target_row['nFlowsProcessedPerMinuteTotals_max'] / 1e6 + + ax6.scatter(prev_flows_per_min, prev_mem, c=color_prev, alpha=0.6, s=50, label='Previous Runs') + ax6.scatter(curr_flows_per_min, curr_mem, c=color_current, s=200, marker='*', + edgecolor='black', linewidth=2, label='Current (Updated)', zorder=5) + + # Add efficiency line (flows per minute per MB) + if prev_flows_per_min: + x_range = np.linspace(min(prev_flows_per_min), max(max(prev_flows_per_min), curr_flows_per_min), 100) + # Calculate average efficiency using flows per minute per MB + efficiencies = [f / m for f, m in zip(prev_flows_per_min, prev_mem)] + avg_efficiency = sum(efficiencies) / len(efficiencies) + y_line = x_range / avg_efficiency + ax6.plot(x_range, y_line, '--', color='gray', alpha=0.5, linewidth=1, + label=f'Last {len(previous_runs)} Avg Efficiency: {avg_efficiency:.2f}M flows/min/MB') + + ax6.set_xlabel('Flows Per Minute (Millions)', fontsize=11, fontweight='bold') + ax6.set_ylabel('Memory Usage RSS (MB)', fontsize=11, fontweight='bold') + ax6.set_title('Memory Efficiency (Flows/Min per MB)', fontsize=13, fontweight='bold') + ax6.grid(True, alpha=0.3) + ax6.legend(fontsize=9) + + # 7. Efficiency Comparison (Percentage) - Full width + ax7 = fig.add_subplot(gs[3, :]) + if previous_runs: + curr_flows_per_min = target_row['nFlowsProcessedPerMinuteTotals_max'] + curr_cpu = target_row['cpuEBPFTotals_avg'] + curr_mem = target_row['rssEBPFTotals_avg'] + + # Calculate current efficiencies + curr_cpu_eff = curr_flows_per_min / curr_cpu / 1e6 if curr_cpu > 0 else 0 # M flows/min per core + curr_mem_eff = curr_flows_per_min / (curr_mem / 1e6) / 1e6 if curr_mem > 0 else 0 # M flows/min per MB + + # Calculate efficiency changes for each individual run + cpu_eff_changes = [] + mem_eff_changes = [] + run_labels = [] + + for prev_run in previous_runs: + prev_flows_per_min = prev_run['nFlowsProcessedPerMinuteTotals_max'] + prev_cpu = prev_run['cpuEBPFTotals_avg'] + prev_mem = prev_run['rssEBPFTotals_avg'] if prev_run['rssEBPFTotals_avg'] > 0 else 0 + + # Calculate previous run efficiencies + prev_cpu_eff = prev_flows_per_min / prev_cpu / 1e6 if prev_cpu > 0 else 0 + prev_mem_eff = prev_flows_per_min / (prev_mem / 1e6) / 1e6 if prev_mem > 0 else 0 + + # Calculate percentage changes + cpu_eff_change = ((curr_cpu_eff - prev_cpu_eff) / prev_cpu_eff * 100) if prev_cpu_eff > 0 else 0 + mem_eff_change = ((curr_mem_eff - prev_mem_eff) / prev_mem_eff * 100) if prev_mem_eff > 0 else 0 + + cpu_eff_changes.append(cpu_eff_change) + mem_eff_changes.append(mem_eff_change) + # Use prow ID as label (full ID, no truncation) + prow_id = prev_run.get('prow_id', 'N/A') + run_labels.append(str(prow_id) if prow_id else 'N/A') + + # Create grouped bar chart + x = np.arange(len(run_labels)) # One group per run + width = 0.35 # Width of bars + + # Create bars for CPU and Memory efficiency + bars_cpu = ax7.bar(x - width/2, cpu_eff_changes, width, + label='CPU Efficiency (flows/min/core)', + color=color_prev, alpha=0.7, edgecolor='black', linewidth=1.5) + bars_mem = ax7.bar(x + width/2, mem_eff_changes, width, + label='Memory Efficiency (flows/min/MB)', + color=color_avg, alpha=0.7, edgecolor='black', linewidth=1.5) + + # Add value labels on bars + for bars in [bars_cpu, bars_mem]: + for bar in bars: + height = bar.get_height() + text_y = height / 2 + text_color = 'white' if abs(height) > 1 else 'black' + ax7.text(bar.get_x() + bar.get_width()/2., text_y, + f'{height:+.2f}%', + ha='center', va='center', + fontweight='bold', fontsize=10, color=text_color) + + # Add zero line + ax7.axhline(y=0, color='black', linestyle='-', linewidth=1) + ax7.set_ylabel('Efficiency Change (%)', fontsize=11, fontweight='bold') + ax7.set_title(f'Efficiency Change vs Each of Last {len(previous_runs)} Runs', fontsize=13, fontweight='bold') + ax7.set_xticks(x) + ax7.set_xticklabels(run_labels, fontsize=8, rotation=0) + ax7.grid(True, alpha=0.3, axis='y') + ax7.legend(fontsize=9, loc='upper right') + + # 8. Summary Statistics - 3 columns + # Create a sub-gridspec for row 4 to split into 3 columns + summary_gs = gs[4, :].subgridspec(1, 3, wspace=0.2) + + ax8 = fig.add_subplot(summary_gs[0, 0]) + ax8.axis('off') + + ax9 = fig.add_subplot(summary_gs[0, 1]) + ax9.axis('off') + + ax10 = fig.add_subplot(summary_gs[0, 2]) + ax10.axis('off') + + if previous_runs: + prev_avg_flows = sum(r['nFlowsProcessedTotals_max'] for r in previous_runs) / len(previous_runs) + prev_avg_flows_per_min = sum(r['nFlowsProcessedPerMinuteTotals_max'] for r in previous_runs) / len(previous_runs) + prev_avg_cpu = sum(r['cpuEBPFTotals_avg'] for r in previous_runs) / len(previous_runs) + prev_avg_mem = sum(r['rssEBPFTotals_avg'] for r in previous_runs if r['rssEBPFTotals_avg'] > 0) / max(1, len([r for r in previous_runs if r['rssEBPFTotals_avg'] > 0])) + prev_avg_dropped = sum(r['ebpfFlowsDroppedRatio_avg'] for r in previous_runs) / len(previous_runs) + + curr_flows = target_row['nFlowsProcessedTotals_max'] + curr_flows_per_min = target_row['nFlowsProcessedPerMinuteTotals_max'] + curr_cpu = target_row['cpuEBPFTotals_avg'] + curr_mem = target_row['rssEBPFTotals_avg'] + curr_dropped = target_row['ebpfFlowsDroppedRatio_avg'] + + flows_change = ((curr_flows - prev_avg_flows) / prev_avg_flows * 100) if prev_avg_flows > 0 else 0 + flows_per_min_change = ((curr_flows_per_min - prev_avg_flows_per_min) / prev_avg_flows_per_min * 100) if prev_avg_flows_per_min > 0 else 0 + cpu_change = ((curr_cpu - prev_avg_cpu) / prev_avg_cpu * 100) if prev_avg_cpu > 0 else 0 + mem_change = ((curr_mem - prev_avg_mem) / prev_avg_mem * 100) if prev_avg_mem > 0 else 0 + dropped_change = ((curr_dropped - prev_avg_dropped) / prev_avg_dropped * 100) if prev_avg_dropped > 0 else (0 if curr_dropped == 0 else float('inf')) + + # Calculate efficiencies for summary + prev_cpu_eff = prev_avg_flows_per_min / prev_avg_cpu / 1e6 if prev_avg_cpu > 0 else 0 + curr_cpu_eff = curr_flows_per_min / curr_cpu / 1e6 if curr_cpu > 0 else 0 + prev_mem_eff = prev_avg_flows_per_min / (prev_avg_mem / 1e6) / 1e6 if prev_avg_mem > 0 else 0 # M flows/min per MB + curr_mem_eff = curr_flows_per_min / (curr_mem / 1e6) / 1e6 if curr_mem > 0 else 0 + + cpu_eff_change = ((curr_cpu_eff - prev_cpu_eff) / prev_cpu_eff * 100) if prev_cpu_eff > 0 else 0 + mem_eff_change = ((curr_mem_eff - prev_mem_eff) / prev_mem_eff * 100) if prev_mem_eff > 0 else 0 + + # Determine dropped flows status + if curr_dropped == 0 and prev_avg_dropped == 0: + dropped_status = '[OK] Zero drops' + elif curr_dropped == 0 and prev_avg_dropped > 0: + dropped_status = '[OK] Eliminated drops' + elif curr_dropped > 0: + dropped_status = '[WARN] Drops present' + else: + dropped_status = '[OK] No change' + + # Format dropped change + if dropped_change == float('inf'): + dropped_change_str = 'N/A' + else: + dropped_change_str = f'{dropped_change:+.2f}%' + + summary_col1 = f""" +PERFORMANCE SUMMARY + +Target Prow ID: {target_prow_id} +Comparing to: Last {len(previous_runs)} runs + +FLOWS PROCESSED + Last {len(previous_runs)} Avg: {prev_avg_flows/1e6:.2f}M flows + Current: {curr_flows/1e6:.2f}M flows + Change: {flows_change:+.2f}% + +FLOWS PER MINUTE + Last {len(previous_runs)} Avg: {prev_avg_flows_per_min/1e6:.2f}M flows/min + Current: {curr_flows_per_min/1e6:.2f}M flows/min + Change: {flows_per_min_change:+.2f}% + """ + + summary_col2 = f""" +RESOURCE USAGE + +CPU USAGE + Last {len(previous_runs)} Avg: {prev_avg_cpu:.3f} cores + Current: {curr_cpu:.3f} cores + Change: {cpu_change:+.2f}% + +MEMORY (RSS) + Last {len(previous_runs)} Avg: {prev_avg_mem/1e9:.2f} GB + Current: {curr_mem/1e9:.2f} GB + Change: {mem_change:+.2f}% + """ + + summary_col3 = f""" +EFFICIENCY (Rate-Based) + +CPU Efficiency: + Last {len(previous_runs)} Avg: {prev_cpu_eff:.2f}M flows/min/core + Current: {curr_cpu_eff:.2f}M flows/min/core + Change: {cpu_eff_change:+.2f}% + +Memory Efficiency: + Last {len(previous_runs)} Avg: {prev_mem_eff:.2f}M flows/min/MB + Current: {curr_mem_eff:.2f}M flows/min/MB + Change: {mem_eff_change:+.2f}% + """ + + ax8.text(0.05, 0.5, summary_col1, fontsize=10, family='monospace', + verticalalignment='center', horizontalalignment='left', + bbox=dict(boxstyle='round', facecolor='wheat', alpha=0.8)) + + ax9.text(0.05, 0.5, summary_col2, fontsize=10, family='monospace', + verticalalignment='center', horizontalalignment='left', + bbox=dict(boxstyle='round', facecolor='wheat', alpha=0.8)) + + ax10.text(0.05, 0.5, summary_col3, fontsize=10, family='monospace', + verticalalignment='center', horizontalalignment='left', + bbox=dict(boxstyle='round', facecolor='wheat', alpha=0.8)) + + # 9. Dropped Flows - Separate text box + ax11 = fig.add_subplot(gs[5, :]) # Full width + ax11.axis('off') + + if previous_runs: + dropped_flows_text = f""" +DROPPED FLOWS (Ratio) + Last {len(previous_runs)} Avg: {prev_avg_dropped:.6f} + Current: {curr_dropped:.6f} + Change: {dropped_change_str} + Status: {dropped_status} + """ + + ax11.text(0.5, 0.5, dropped_flows_text, fontsize=10, family='monospace', + verticalalignment='center', horizontalalignment='center', + bbox=dict(boxstyle='round', facecolor='lightgreen' if dropped_status.startswith('[OK]') else 'lightyellow', alpha=0.8)) + + # Main title + fig.suptitle('eBPF Agent Performance Analysis\n(Updated Code vs Previous Runs)', + fontsize=16, fontweight='bold', y=0.98) + + # Determine output file + if output_file is None: + # Create perf directory if it doesn't exist + perf_dir = 'perf' + os.makedirs(perf_dir, exist_ok=True) + output_file = os.path.join(perf_dir, 'ebpf_performance_visualization.png') + else: + # If output is specified, ensure directory exists + output_dir = os.path.dirname(output_file) if os.path.dirname(output_file) else '.' + if output_dir and output_dir != '.': + os.makedirs(output_dir, exist_ok=True) + + plt.savefig(output_file, dpi=300, bbox_inches='tight', facecolor='white') + print(f"[OK] Visualization saved to: {output_file}") + return output_file + +def main(): + """Main entry point.""" + parser = argparse.ArgumentParser( + description='Visualize eBPF agent performance data from CSV file', + formatter_class=argparse.RawDescriptionHelpFormatter, + epilog=""" +Examples: + %(prog)s data.csv 1985348508604960768 + %(prog)s data.csv 1985348508604960768 --output perf.png + %(prog)s data.csv 1985348508604960768 --num-runs 5 + """ + ) + parser.add_argument('csv_file', help='Path to CSV file with performance data') + parser.add_argument('prow_id', help='Prow ID of the target run to compare') + parser.add_argument('--output', '-o', help='Output PNG file path (default: perf/ebpf_performance_visualization.png)') + parser.add_argument('--num-runs', '-n', type=int, default=3, + help='Number of previous runs to compare against (default: 3)') + + args = parser.parse_args() + + # Check if CSV file exists + if not os.path.exists(args.csv_file): + print(f"Error: CSV file not found: {args.csv_file}") + sys.exit(1) + + # Validate num_runs + if args.num_runs < 1: + print(f"Error: --num-runs must be at least 1") + sys.exit(1) + + print(f"Creating visualization from {args.csv_file}") + print(f"Target Prow ID: {args.prow_id}") + print(f"Comparing against last {args.num_runs} previous runs") + + visualize_csv_data(args.csv_file, args.prow_id, args.output, args.num_runs) + print("\n[OK] Visualization completed successfully!") + +if __name__ == '__main__': + main() +