diff --git a/README.md b/README.md index 53565d3..32cdc8d 100644 --- a/README.md +++ b/README.md @@ -43,6 +43,7 @@ processing operations. These building blocks can be used to transform and manipu - **Flatten1:** Flattens a stream of slices of elements into a stream of elements. - **Batch:** Breaks a stream of elements into batches based on size or timing. - **Throttler:** Limits the rate at which elements are processed. +- **AdaptiveThrottler:** Limits the rate at which elements are processed based on the current system resource utilization (CPU and memory usage). - **SlidingWindow:** Creates overlapping windows of elements. - **TumblingWindow:** Creates non-overlapping, fixed-size windows of elements. - **SessionWindow:** Creates windows based on periods of activity and inactivity. diff --git a/examples/adaptive_throttler/demo/demo.go b/examples/adaptive_throttler/demo/demo.go new file mode 100644 index 0000000..8a8164e --- /dev/null +++ b/examples/adaptive_throttler/demo/demo.go @@ -0,0 +1,154 @@ +package main + +import ( + "fmt" + "math/rand" + "sync/atomic" + "time" + + ext "github.com/reugn/go-streams/extension" + "github.com/reugn/go-streams/flow" +) + +// main demonstrates adaptive throttling with simulated resource pressure. +// The demo: +// 1. Produces 250 elements in bursts +// 2. Processes elements with CPU-intensive work (50ms each) +// 3. Simulates memory pressure that increases then decreases (creating throttling-recovery cycle) +// 4. The adaptive throttler adjusts throughput based on CPU/memory usage +// 5. Shows throttling down to ~1/sec during high memory, then recovery back to 40/sec +// 6. Stats are logged every 500ms showing rate adaptation +func main() { + var elementsProcessed atomic.Int64 + + // Set up demo configuration with memory simulation + throttler := setupDemoThrottler(&elementsProcessed) + + in := make(chan any) + out := make(chan any) // Unbuffered channel to prevent apparent bursts + + source := ext.NewChanSource(in) + sink := ext.NewChanSink(out) + + statsDone := make(chan struct{}) + go logThrottlerStats(throttler, statsDone) + defer close(statsDone) + + go func() { + source. + Via(throttler). + Via(flow.NewPassThrough()). + To(sink) + }() + + go produceBurst(in, 250) + + var cpuWorkChecksum uint64 + + // Process the output + elementsReceived := 0 + for element := range sink.Out { + fmt.Printf("consumer received %v\n", element) + elementsProcessed.Add(1) + elementsReceived++ + + // Perform CPU-intensive work + burnCPU(50*time.Millisecond, &cpuWorkChecksum) + + time.Sleep(25 * time.Millisecond) + } + + fmt.Printf("CPU work checksum: %d\n", cpuWorkChecksum) + fmt.Printf("Total elements produced: 250, Total elements received: %d\n", elementsReceived) + if elementsReceived == 250 { + fmt.Println("✅ SUCCESS: All elements processed without dropping!") + } else { + fmt.Printf("❌ FAILURE: %d elements were dropped!\n", 250-elementsReceived) + } + + throttler.Close() + + fmt.Println("adaptive throttling pipeline completed") +} + +// setupDemoThrottler creates and configures an adaptive throttler with demo settings +func setupDemoThrottler(elementsProcessed *atomic.Int64) *flow.AdaptiveThrottler { + config := flow.DefaultAdaptiveThrottlerConfig() + + config.MinRate = 1 + config.MaxRate = 20 + config.InitialRate = 20 + config.SampleInterval = 200 * time.Millisecond + + config.BackoffFactor = 0.5 + config.RecoveryFactor = 1.5 + + config.MaxMemoryPercent = 35.0 + config.RecoveryMemoryThreshold = 30.0 + + // Memory Reader Simulation - Creates a cycle: low -> high -> low memory usage + config.MemoryReader = func() (float64, error) { + elementCount := elementsProcessed.Load() + + var memoryPercent float64 + switch { + case elementCount <= 80: // Phase 1: Low memory, allow high throughput + memoryPercent = 5.0 + float64(elementCount)*0.1 // 5% to 13% + case elementCount <= 120: // Phase 2: Increasing memory pressure, cause throttling + memoryPercent = 15.0 + float64(elementCount-80)*0.6 // 15% to 43% + case elementCount <= 160: // Phase 3: High memory, keep throttled down to ~1/sec + memoryPercent = 30.0 + float64(elementCount-120)*0.3 // 30% to 42% + default: // Phase 4: Memory decreases, allow recovery back to 40/sec + memoryPercent = 25.0 - float64(elementCount-160)*1.5 // 25% down to ~5% + if memoryPercent < 5.0 { + memoryPercent = 5.0 + } + } + return memoryPercent, nil + } + + throttler, err := flow.NewAdaptiveThrottler(config) + if err != nil { + panic(fmt.Sprintf("failed to create adaptive throttler: %v", err)) + } + return throttler +} + +func produceBurst(in chan<- any, total int) { + defer close(in) + + for i := range total { + in <- fmt.Sprintf("job-%02d", i) + + if (i+1)%10 == 0 { + time.Sleep(180 * time.Millisecond) + continue + } + time.Sleep(time.Duration(2+rand.Intn(5)) * time.Millisecond) + } +} + +func burnCPU(duration time.Duration, checksum *uint64) { + start := time.Now() + for time.Since(start) < duration { + for i := 0; i < 1000; i++ { + *checksum += uint64(i * i) + } + } +} + +func logThrottlerStats(at *flow.AdaptiveThrottler, done <-chan struct{}) { + ticker := time.NewTicker(500 * time.Millisecond) + defer ticker.Stop() + + for { + select { + case <-done: + return + case <-ticker.C: + stats := at.GetResourceStats() + fmt.Printf("[stats] Rate: %.1f/sec, CPU: %.1f%%, Memory: %.1f%%\n", + at.GetCurrentRate(), stats.CPUUsagePercent, stats.MemoryUsedPercent) + } + } +} diff --git a/examples/adaptive_throttler/main.go b/examples/adaptive_throttler/main.go new file mode 100644 index 0000000..36952bf --- /dev/null +++ b/examples/adaptive_throttler/main.go @@ -0,0 +1,150 @@ +package main + +import ( + "fmt" + "math/rand" + "strings" + "sync/atomic" + "time" + + ext "github.com/reugn/go-streams/extension" + "github.com/reugn/go-streams/flow" +) + +// Demo of adaptive throttling with CPU-intensive work. +// T1 throttles on CPU > 0.01%, T2 throttles on Memory > 50%. + +func editMessage(msg string) string { + return strings.ToUpper(msg) +} + +func addTimestamp(msg string) string { + return fmt.Sprintf("[%s] %s", time.Now().Format("15:04:05"), msg) +} + +// cpuIntensiveWork simulates processing load that affects CPU usage +func cpuIntensiveWork(msg string) string { + // Simulate CPU-intensive work (hashing-like computation) + var checksum uint64 + for i := 0; i < 200000; i++ { // Increased from 50000 to 200000 for more CPU load + checksum += uint64(len(msg)) * uint64(i) //nolint:gosec + } + return msg +} + +func main() { + var messagesProcessed atomic.Int64 + + // Configure first throttler - CPU-focused with higher initial rate + throttler1Config := flow.DefaultAdaptiveThrottlerConfig() + throttler1Config.MaxCPUPercent = 0.01 // Throttle when CPU > 0.01% (extremely low threshold) + throttler1Config.MaxMemoryPercent = 80.0 // Less strict memory limit + throttler1Config.InitialRate = 50 // Start at 50/sec + throttler1Config.MaxRate = 100 // Can go up to 100/sec + throttler1Config.MinRate = 5 // Minimum 5/sec + throttler1Config.SampleInterval = 200 * time.Millisecond + throttler1Config.BackoffFactor = 0.6 // Reduce to 60% when constrained + throttler1Config.RecoveryFactor = 1.4 // Increase by 40% during recovery + + throttler1, err := flow.NewAdaptiveThrottler(throttler1Config) + if err != nil { + panic(fmt.Sprintf("failed to create throttler1: %v", err)) + } + + // Configure second throttler - Memory-focused with memory simulation + throttler2Config := flow.DefaultAdaptiveThrottlerConfig() + throttler2Config.MaxCPUPercent = 80.0 // Less strict CPU limit + throttler2Config.MaxMemoryPercent = 40.0 // Throttle when memory > 40% + throttler2Config.InitialRate = 30 // Start at 30/sec + throttler2Config.MaxRate = 80 // Can go up to 80/sec + throttler2Config.MinRate = 3 // Minimum 3/sec + throttler2Config.SampleInterval = 200 * time.Millisecond + throttler2Config.BackoffFactor = 0.5 // Reduce to 50% when constrained + throttler2Config.RecoveryFactor = 1.3 // Increase by 30% during recovery + + // Use system memory but with lower threshold to demonstrate throttling + throttler2Config.MaxMemoryPercent = 50.0 // Lower threshold than T1 + + throttler2, err := flow.NewAdaptiveThrottler(throttler2Config) + if err != nil { + panic(fmt.Sprintf("failed to create throttler2: %v", err)) + } + + in := make(chan any) + + source := ext.NewChanSource(in) + editMapFlow := flow.NewMap(editMessage, 1) + cpuWorkFlow := flow.NewMap(cpuIntensiveWork, 1) // Add CPU-intensive work + timestampFlow := flow.NewMap(addTimestamp, 1) + sink := ext.NewStdoutSink() + + // Pipeline: Source -> Throttler1 (CPU) -> CPU Work -> Throttler2 (Memory) -> Edit -> Timestamp -> Sink + go func() { + source. + Via(throttler1). // First throttler monitors CPU + Via(cpuWorkFlow). // CPU-intensive processing + Via(throttler2). // Second throttler monitors memory + Via(editMapFlow). // Simple transformation + Via(timestampFlow). // Add timestamp + To(sink) + }() + + // Enhanced stats logging showing throttling behavior + go func() { + ticker := time.NewTicker(500 * time.Millisecond) + defer ticker.Stop() + + for range ticker.C { + stats1 := throttler1.GetResourceStats() + stats2 := throttler2.GetResourceStats() + + // Show current rates and resource usage + fmt.Printf("[stats] T1-CPU: %.1f/s (CPU:%.1f%%), T2-Mem: %.1f/s (Mem:%.1f%%), Total: %d msgs\n", + throttler1.GetCurrentRate(), + stats1.CPUUsagePercent, + throttler2.GetCurrentRate(), + stats2.MemoryUsedPercent, + messagesProcessed.Load()) + + // Show throttling status + throttleReason := "" + if stats1.CPUUsagePercent > throttler1Config.MaxCPUPercent { + throttleReason += "T1:CPU-high " + } + if stats2.MemoryUsedPercent > throttler2Config.MaxMemoryPercent { + throttleReason += "T2:Mem-high " + } + if throttleReason == "" { + throttleReason = "No throttling" + } + fmt.Printf("[throttle] %s\n", throttleReason) + } + }() + + // Producer with bursty traffic to test throttling + go func() { + defer close(in) + + for i := 1; i <= 100; i++ { + message := fmt.Sprintf("MESSAGE-%d", i) + in <- message + messagesProcessed.Add(1) + + // Variable delay to create bursts (some fast, some slow) + var delay time.Duration + if i%20 == 0 { + delay = 100 * time.Millisecond // Burst pause every 20 messages + } else { + delay = time.Duration(5+rand.Intn(15)) * time.Millisecond //nolint:gosec // 5-20ms between messages + } + time.Sleep(delay) + } + }() + + sink.AwaitCompletion() + + throttler1.Close() + throttler2.Close() + + fmt.Printf("Demo completed! Processed %d messages\n", messagesProcessed.Load()) +} diff --git a/examples/go.mod b/examples/go.mod index a2b191f..a203164 100644 --- a/examples/go.mod +++ b/examples/go.mod @@ -166,6 +166,7 @@ require ( ) replace ( + github.com/reugn/go-streams => .. github.com/reugn/go-streams/aerospike => ../aerospike github.com/reugn/go-streams/aws => ../aws github.com/reugn/go-streams/azure => ../azure diff --git a/examples/go.sum b/examples/go.sum index 391f03e..02982e0 100644 --- a/examples/go.sum +++ b/examples/go.sum @@ -361,8 +361,6 @@ github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5X github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/redis/go-redis/v9 v9.8.0 h1:q3nRvjrlge/6UD7eTu/DSg2uYiU2mCL0G/uzBWqhicI= github.com/redis/go-redis/v9 v9.8.0/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= -github.com/reugn/go-streams v0.12.0 h1:SEfTZw5+iP0mQG0jWTxVOHMbqlbZjUB0W6dF3XbYd5Q= -github.com/reugn/go-streams v0.12.0/go.mod h1:dnXv6QgVTW62gEpILoLHRjI95Es7ECK2/+j9h17aIN8= github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= github.com/shirou/gopsutil/v3 v3.23.12 h1:z90NtUkp3bMtmICZKpC4+WaknU1eXtp5vtbQ11DgpE4= diff --git a/flow/adaptive_throttler.go b/flow/adaptive_throttler.go new file mode 100644 index 0000000..9af7de3 --- /dev/null +++ b/flow/adaptive_throttler.go @@ -0,0 +1,362 @@ +package flow + +import ( + "fmt" + "math" + "sync/atomic" + "time" + + "github.com/reugn/go-streams" +) + +const ( + // minSampleInterval is the minimum allowed sampling interval + minSampleInterval = 50 * time.Millisecond + // smoothingFactor factor by which the current rate is adjusted when smoothing is enabled + smoothingFactor = 0.3 +) + +// AdaptiveThrottlerConfig configures the adaptive throttler behavior +type AdaptiveThrottlerConfig struct { + // MaxMemoryPercent is the maximum allowed memory usage percentage (0-100). + // When memory usage exceeds this threshold, the throttler reduces processing rate. + // Default config value: 85.0 + MaxMemoryPercent float64 + + // MaxCPUPercent is the maximum allowed CPU usage percentage (0-100). + // When CPU usage exceeds this threshold, the throttler reduces processing rate. + // Default config value: 80.0 + MaxCPUPercent float64 + + // Sampling configuration + // SampleInterval is the frequency of resource monitoring and rate adjustments. + // Minimum: 50ms. Default config value: 100ms + SampleInterval time.Duration + + // CPUUsageMode specifies the CPU monitoring strategy. + // Options: CPUUsageModeHeuristic (goroutine-based) or CPUUsageModeMeasured (system calls). + // Default config value: CPUUsageModeMeasured + CPUUsageMode CPUUsageMode + + // MemoryReader is an optional custom function to read memory usage percentage (0-100). + // If nil, system memory usage is monitored automatically. + // Function signature: func() (float64, error) - returns percentage and any error. + MemoryReader func() (float64, error) + + // Rate Limits + // InitialRate is the starting processing rate in items per second. + // Must be between MinRate and MaxRate. Default config value: 1000 + InitialRate int + + // MinRate is the minimum allowed processing rate in items per second. + // Must be > 0. Default config value: 10 + MinRate int + + // MaxRate is the maximum allowed processing rate in items per second. + // Must be > MinRate. Default config value: 10000 + MaxRate int + + // Adjustment factors + // BackoffFactor is the multiplier applied to reduce rate when resource constraints are exceeded. + // Range: 0.0-1.0 (e.g., 0.7 means rate is reduced to 70% when constrained). + // Default config value: 0.7 + BackoffFactor float64 + + // RecoveryCPUThreshold is the CPU usage percentage below which rate recovery begins. + // Must be < MaxCPUPercent. If zero, defaults to MaxCPUPercent-10 or 90% of MaxCPUPercent. + // Range: 0.0-MaxCPUPercent + RecoveryCPUThreshold float64 + + // RecoveryMemoryThreshold is the memory usage percentage below which rate recovery begins. + // Must be < MaxMemoryPercent. If zero, defaults to MaxMemoryPercent-10 or 90% of MaxMemoryPercent. + // Range: 0.0-MaxMemoryPercent + RecoveryMemoryThreshold float64 + + // RecoveryFactor is the multiplier applied to increase rate during recovery periods. + // Must be > 1.0 (e.g., 1.3 means rate increases by 30% during recovery). + // Default config value: 1.3 + RecoveryFactor float64 + + // EnableHysteresis prevents rapid rate oscillations by requiring both resource recovery + // thresholds to be met before increasing rate. When false, rate increases as soon as + // constraints are removed. + // Default config value: true + EnableHysteresis bool +} + +// DefaultAdaptiveThrottlerConfig returns safe defaults +func DefaultAdaptiveThrottlerConfig() *AdaptiveThrottlerConfig { + return &AdaptiveThrottlerConfig{ + MaxMemoryPercent: 85.0, + MaxCPUPercent: 80.0, + SampleInterval: 100 * time.Millisecond, + CPUUsageMode: CPUUsageModeMeasured, + InitialRate: 1000, + MinRate: 10, + MaxRate: 10000, + BackoffFactor: 0.7, + RecoveryFactor: 1.3, + EnableHysteresis: true, + } +} + +func (c *AdaptiveThrottlerConfig) validate() error { + if c.SampleInterval < minSampleInterval { + return fmt.Errorf("sample interval must be at least %v", minSampleInterval) + } + if c.MaxMemoryPercent <= 0 || c.MaxMemoryPercent > 100 { + return fmt.Errorf("MaxMemoryPercent must be between 0 and 100") + } + if c.MaxCPUPercent < 0 || c.MaxCPUPercent > 100 { + return fmt.Errorf("MaxCPUPercent must be between 0 and 100") + } + + // Set default recovery thresholds if not specified + if c.RecoveryMemoryThreshold == 0 { + c.RecoveryMemoryThreshold = c.MaxMemoryPercent - 10 + if c.RecoveryMemoryThreshold < 0 { + c.RecoveryMemoryThreshold = c.MaxMemoryPercent * 0.9 // 90% of max if max < 10 + } + } + if c.RecoveryCPUThreshold == 0 { + c.RecoveryCPUThreshold = c.MaxCPUPercent - 10 + if c.RecoveryCPUThreshold < 0 { + c.RecoveryCPUThreshold = c.MaxCPUPercent * 0.9 // 90% of max if max < 10 + } + } + + // Validate recovery thresholds + if c.RecoveryMemoryThreshold < 0 || c.RecoveryMemoryThreshold >= c.MaxMemoryPercent { + return fmt.Errorf("RecoveryMemoryThreshold (%.1f) must be between 0 and MaxMemoryPercent (%.1f)", + c.RecoveryMemoryThreshold, c.MaxMemoryPercent) + } + if c.MaxCPUPercent > 0 && (c.RecoveryCPUThreshold < 0 || c.RecoveryCPUThreshold >= c.MaxCPUPercent) { + return fmt.Errorf("RecoveryCPUThreshold (%.1f) must be between 0 and MaxCPUPercent (%.1f)", + c.RecoveryCPUThreshold, c.MaxCPUPercent) + } + + if c.BackoffFactor >= 1.0 || c.BackoffFactor <= 0 { + return fmt.Errorf("BackoffFactor must be between 0 and 1") + } + if c.MinRate <= 0 { + return fmt.Errorf("MinRate must be greater than 0") + } + if c.MaxRate <= c.MinRate { + return fmt.Errorf("MaxRate must be greater than MinRate") + } + if c.InitialRate < c.MinRate || c.InitialRate > c.MaxRate { + return fmt.Errorf("InitialRate must be between MinRate (%d) and MaxRate (%d) inclusive", c.MinRate, c.MaxRate) + } + if c.RecoveryFactor <= 1.0 { + return fmt.Errorf("RecoveryFactor must be greater than 1") + } + return nil +} + +// AdaptiveThrottler implements a feedback control system that monitors resources +// and adjusts throughput dynamically using a token bucket with adjustable rate. +type AdaptiveThrottler struct { + config AdaptiveThrottlerConfig + monitor resourceMonitor + currentRateBits uint64 + + in chan any + out chan any + done chan struct{} + closed atomic.Bool +} + +// NewAdaptiveThrottler creates a new instance +func NewAdaptiveThrottler(config *AdaptiveThrottlerConfig) (*AdaptiveThrottler, error) { + if config == nil { + config = DefaultAdaptiveThrottlerConfig() + } + if err := config.validate(); err != nil { + return nil, err + } + + monitor := globalMonitorRegistry.Acquire( + config.SampleInterval, + config.CPUUsageMode, + config.MemoryReader, + ) + + at := &AdaptiveThrottler{ + config: *config, + monitor: monitor, + in: make(chan any), + out: make(chan any), + done: make(chan struct{}), + } + + // Set initial rate atomically + at.setRate(float64(config.InitialRate)) + + // Start the monitor loop to adjust the rate based on the resource usage + go at.monitorLoop() + + // Start the pipeline loop to emit items at the correct rate + go at.pipelineLoop() + + return at, nil +} + +// Via asynchronously streams data to the given Flow and returns it. +func (at *AdaptiveThrottler) Via(flow streams.Flow) streams.Flow { + go at.streamPortioned(flow) + return flow +} + +// To streams data to the given Sink and blocks until the Sink has completed +// processing all data. +func (at *AdaptiveThrottler) To(sink streams.Sink) { + at.streamPortioned(sink) + sink.AwaitCompletion() +} + +func (at *AdaptiveThrottler) Out() <-chan any { + return at.out +} + +func (at *AdaptiveThrottler) In() chan<- any { + return at.in +} + +// setRate updates the current rate limit atomically +func (at *AdaptiveThrottler) setRate(rate float64) { + atomic.StoreUint64(&at.currentRateBits, math.Float64bits(rate)) +} + +// GetCurrentRate returns the current rate limit atomically +func (at *AdaptiveThrottler) GetCurrentRate() float64 { + return math.Float64frombits(atomic.LoadUint64(&at.currentRateBits)) +} + +// GetResourceStats returns the latest resource statistics from the monitor +func (at *AdaptiveThrottler) GetResourceStats() ResourceStats { + return at.monitor.GetStats() +} + +// streamPortioned streams elements to the given Inlet from the throttler's output. +// Elements are sent to inlet.In() until at.out is closed. +func (at *AdaptiveThrottler) streamPortioned(inlet streams.Inlet) { + defer close(inlet.In()) + for element := range at.Out() { + select { + case inlet.In() <- element: + case <-at.done: + // Throttler was closed, exit early + return + } + } +} + +func (at *AdaptiveThrottler) Close() { + if at.closed.CompareAndSwap(false, true) { + close(at.done) + at.monitor.Close() + } +} + +// monitorLoop handles periodic resource checks and rate adjustment. +func (at *AdaptiveThrottler) monitorLoop() { + ticker := time.NewTicker(at.config.SampleInterval) + defer ticker.Stop() + + for { + select { + case <-at.done: + return + case <-ticker.C: + at.adjustRate() + } + } +} + +// Pipeline Loop: Strict Pacer +// Ensures exactly 1/Rate seconds between emissions. +func (at *AdaptiveThrottler) pipelineLoop() { + defer close(at.out) + + // The earliest allowed time for the next item to be emitted + nextEmission := time.Now() + + for item := range at.in { + // Check if the throttler is done + select { + case <-at.done: + return + default: + } + + // Get the current interval + rate := at.GetCurrentRate() + if rate < 1.0 { + rate = 1.0 + } + + // Calculate the interval between emissions for the current rate + interval := time.Duration(float64(time.Second) / rate) + + now := time.Now() + + // If we are ahead of schedule, sleep until the next emission + if now.Before(nextEmission) { + sleepDuration := nextEmission.Sub(now) + select { + case <-time.After(sleepDuration): + now = time.Now() + case <-at.done: + return + } + } + + nextEmission = now.Add(interval) + + // Emit + select { + case at.out <- item: + case <-at.done: + return + } + } +} + +// adjustRate calculates the new rate based on stats and updates it atomically. +func (at *AdaptiveThrottler) adjustRate() { + stats := at.monitor.GetStats() + currentRate := at.GetCurrentRate() + + // Check if the resource usage is above the threshold + isConstrained := stats.MemoryUsedPercent > at.config.MaxMemoryPercent || + stats.CPUUsagePercent > at.config.MaxCPUPercent + + // Check if the resource usage is below the recovery threshold + isBelowRecovery := stats.MemoryUsedPercent < at.config.RecoveryMemoryThreshold && + stats.CPUUsagePercent < at.config.RecoveryCPUThreshold + + // Check if the resource usage is below the recovery threshold and hysteresis is disabled + shouldIncrease := !isConstrained && (!at.config.EnableHysteresis || isBelowRecovery) + + targetRate := currentRate + if isConstrained { + // Reduce the rate by the backoff factor + targetRate *= at.config.BackoffFactor + } else if shouldIncrease { + // Increase the rate by the recovery factor + targetRate *= at.config.RecoveryFactor + if targetRate > float64(at.config.MaxRate) { + targetRate = float64(at.config.MaxRate) + } + } + + // Apply smoothing to the new rate + newRate := currentRate + (targetRate-currentRate)*smoothingFactor + + // Enforce minimum rate + if newRate < float64(at.config.MinRate) { + newRate = float64(at.config.MinRate) + } + + at.setRate(newRate) +} diff --git a/flow/adaptive_throttler_test.go b/flow/adaptive_throttler_test.go new file mode 100644 index 0000000..55c07e1 --- /dev/null +++ b/flow/adaptive_throttler_test.go @@ -0,0 +1,1508 @@ +package flow + +import ( + "math" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/reugn/go-streams/internal/assert" +) + +type MockMonitor struct { + getStatsReturns []ResourceStats + getStatsIndex int + closeCalled bool +} + +func (m *MockMonitor) GetStats() ResourceStats { + if m.getStatsIndex < len(m.getStatsReturns) { + result := m.getStatsReturns[m.getStatsIndex] + m.getStatsIndex++ + return result + } + return ResourceStats{} +} + +func (m *MockMonitor) Close() { + m.closeCalled = true +} + +func (m *MockMonitor) ExpectGetStats(stats ...ResourceStats) { + m.getStatsReturns = stats + m.getStatsIndex = 0 +} + +type mockSink struct { + in chan any + completion chan struct{} + completed atomic.Bool +} + +func (m *mockSink) In() chan<- any { + return m.in +} + +func (m *mockSink) AwaitCompletion() { + if !m.completed.Load() { + if m.completed.CompareAndSwap(false, true) { + if m.completion != nil { + close(m.completion) + } + } + } +} + +type mockSinkWithChannelDrain struct { + in chan any + done chan struct{} +} + +func (m *mockSinkWithChannelDrain) In() chan<- any { + return m.in +} + +func (m *mockSinkWithChannelDrain) AwaitCompletion() { + <-m.done +} + +// newMockSinkWithChannelDrain creates a mock sink that drains the channel like real sinks do +func newMockSinkWithChannelDrain() *mockSinkWithChannelDrain { + sink := &mockSinkWithChannelDrain{ + in: make(chan any), + done: make(chan struct{}), + } + go func() { + defer close(sink.done) + for range sink.in { + _ = struct{}{} + } + }() + return sink +} + +// createThrottlerWithLongInterval creates a throttler with default config but long sample interval +func createThrottlerWithLongInterval(t *testing.T) *AdaptiveThrottler { + t.Helper() + config := DefaultAdaptiveThrottlerConfig() + config.SampleInterval = 10 * time.Second + at, err := NewAdaptiveThrottler(config) + if err != nil { + t.Fatalf("Failed to create throttler: %v", err) + } + return at +} + +// collectDataFromChannel collects all data from a channel into a slice +func collectDataFromChannel(ch <-chan any) []any { + var received []any + for data := range ch { + received = append(received, data) + } + return received +} + +// collectDataFromChannelWithMutex collects data from channel using mutex for thread safety +func collectDataFromChannelWithMutex(ch <-chan any, received *[]any, mu *sync.Mutex, done chan struct{}) { + defer close(done) + for data := range ch { + mu.Lock() + *received = append(*received, data) + mu.Unlock() + } +} + +// verifyChannelClosed checks that a channel is closed within a timeout +func verifyChannelClosed(t *testing.T, ch <-chan any, timeout time.Duration) { + t.Helper() + select { + case _, ok := <-ch: + if ok { + t.Errorf("Channel should be closed") + } + case <-time.After(timeout): + t.Errorf("Channel should be closed within %v", timeout) + } +} + +// createThrottlerForRateTesting creates a throttler with mock monitor for rate adjustment testing +func createThrottlerForRateTesting( + config *AdaptiveThrottlerConfig, + initialRate float64, +) (*AdaptiveThrottler, *MockMonitor) { + mockMonitor := &MockMonitor{} + throttler := &AdaptiveThrottler{ + config: *config, + monitor: mockMonitor, + currentRateBits: math.Float64bits(initialRate), + } + return throttler, mockMonitor +} + +// calculateNewRate calculates the new rate based on current rate and resource stats +// This implements the core adaptive throttling algorithm +func calculateNewRate(config *AdaptiveThrottlerConfig, currentRate float64, stats ResourceStats) float64 { + isConstrained := stats.MemoryUsedPercent > config.MaxMemoryPercent || + stats.CPUUsagePercent > config.MaxCPUPercent + + isBelowRecovery := stats.MemoryUsedPercent < config.RecoveryMemoryThreshold && + stats.CPUUsagePercent < config.RecoveryCPUThreshold + + shouldIncrease := !isConstrained && (!config.EnableHysteresis || isBelowRecovery) + + targetRate := currentRate + if isConstrained { + targetRate *= config.BackoffFactor + } else if shouldIncrease { + targetRate *= config.RecoveryFactor + if targetRate > float64(config.MaxRate) { + targetRate = float64(config.MaxRate) + } + } + + newRate := currentRate + (targetRate-currentRate)*smoothingFactor + + if newRate < float64(config.MinRate) { + newRate = float64(config.MinRate) + } + + return newRate +} + +// simulateRateAdjustments simulates a sequence of rate adjustments and returns the final rate +// This is used to calculate expected rate ranges algorithmically instead of hardcoding them +func simulateRateAdjustments( + config *AdaptiveThrottlerConfig, + initialRate float64, + statsSequence []ResourceStats, +) float64 { + currentRate := initialRate + + for _, stats := range statsSequence { + currentRate = calculateNewRate(config, currentRate, stats) + } + + return currentRate +} + +// calculateExpectedRateRange calculates the expected final rate range for a sustained load scenario +// Returns min and max expected rates based on the algorithm behavior +func calculateExpectedRateRange( + config *AdaptiveThrottlerConfig, + initialRate float64, + statsSequence []ResourceStats, +) (float64, float64) { + finalRate := simulateRateAdjustments(config, initialRate, statsSequence) + + tolerance := 0.1 // 10% tolerance + + minExpected := finalRate * (1 - tolerance) + maxExpected := finalRate * (1 + tolerance) + + if minExpected < float64(config.MinRate) { + minExpected = float64(config.MinRate) + } + if maxExpected > float64(config.MaxRate) { + maxExpected = float64(config.MaxRate) + } + + return minExpected, maxExpected +} + +// calculateExpectedRecoveryRate calculates the expected recovery rate after a high load followed by normal load +// Returns the expected rate after the specified number of recovery cycles +func calculateExpectedRecoveryRate( + config *AdaptiveThrottlerConfig, + initialRate float64, + highLoadStats, normalLoadStats ResourceStats, + recoveryCycles int, +) float64 { + currentRate := initialRate + + currentRate = simulateSingleAdjustment(config, currentRate, highLoadStats) + + for i := 0; i < recoveryCycles+1; i++ { // +1 for the first normal load adjustment + currentRate = simulateSingleAdjustment(config, currentRate, normalLoadStats) + } + + return currentRate +} + +// simulateSingleAdjustment simulates a single rate adjustment for given stats +func simulateSingleAdjustment(config *AdaptiveThrottlerConfig, currentRate float64, stats ResourceStats) float64 { + return calculateNewRate(config, currentRate, stats) +} + +type mockInlet struct { + in chan any +} + +func (m *mockInlet) In() chan<- any { + return m.in +} + +// TestAdaptiveThrottlerConfig_Validate tests configuration validation with valid and invalid settings +func TestAdaptiveThrottlerConfig_Validate(t *testing.T) { + tests := []struct { + name string + config AdaptiveThrottlerConfig + wantErr bool + }{ + { + name: "Valid Config", + config: AdaptiveThrottlerConfig{ + MaxMemoryPercent: 80, + MaxCPUPercent: 70, + SampleInterval: 100 * time.Millisecond, + CPUUsageMode: CPUUsageModeMeasured, + InitialRate: 100, + MinRate: 10, + MaxRate: 1000, + BackoffFactor: 0.5, + RecoveryFactor: 1.2, + }, + wantErr: false, + }, + { + name: "Invalid SampleInterval", + config: AdaptiveThrottlerConfig{ + SampleInterval: 1 * time.Millisecond, + }, + wantErr: true, + }, + { + name: "Invalid MaxMemoryPercent High", + config: AdaptiveThrottlerConfig{ + SampleInterval: 100 * time.Millisecond, + MaxMemoryPercent: 101, + }, + wantErr: true, + }, + { + name: "Invalid BackoffFactor", + config: AdaptiveThrottlerConfig{ + SampleInterval: 100 * time.Millisecond, + MaxMemoryPercent: 80, + BackoffFactor: 1.5, + }, + wantErr: true, + }, + { + name: "Invalid InitialRate below MinRate", + config: AdaptiveThrottlerConfig{ + SampleInterval: 100 * time.Millisecond, + MaxMemoryPercent: 80, + MinRate: 10, + MaxRate: 100, + InitialRate: 5, + BackoffFactor: 0.5, + RecoveryFactor: 1.2, + }, + wantErr: true, + }, + { + name: "Invalid RecoveryFactor too low", + config: AdaptiveThrottlerConfig{ + SampleInterval: 100 * time.Millisecond, + MaxMemoryPercent: 80, + MinRate: 10, + MaxRate: 100, + InitialRate: 50, + BackoffFactor: 0.5, + RecoveryFactor: 0.9, + }, + wantErr: true, + }, + { + name: "Valid InitialRate equals MinRate", + config: AdaptiveThrottlerConfig{ + SampleInterval: 100 * time.Millisecond, + MaxMemoryPercent: 80, + MinRate: 10, + MaxRate: 100, + InitialRate: 10, + BackoffFactor: 0.5, + RecoveryFactor: 1.2, + }, + wantErr: false, + }, + { + name: "Valid InitialRate equals MaxRate", + config: AdaptiveThrottlerConfig{ + SampleInterval: 100 * time.Millisecond, + MaxMemoryPercent: 80, + MinRate: 10, + MaxRate: 100, + InitialRate: 100, + BackoffFactor: 0.5, + RecoveryFactor: 1.2, + }, + wantErr: false, + }, + { + name: "Invalid RecoveryMemoryThreshold too high", + config: AdaptiveThrottlerConfig{ + SampleInterval: 100 * time.Millisecond, + MaxMemoryPercent: 80, + RecoveryMemoryThreshold: 85, // Higher than MaxMemoryPercent + MinRate: 10, + MaxRate: 100, + InitialRate: 50, + BackoffFactor: 0.5, + RecoveryFactor: 1.2, + }, + wantErr: true, + }, + { + name: "Invalid RecoveryCPUThreshold too high", + config: AdaptiveThrottlerConfig{ + SampleInterval: 100 * time.Millisecond, + MaxMemoryPercent: 80, + MaxCPUPercent: 70, + RecoveryCPUThreshold: 75, // Higher than MaxCPUPercent + MinRate: 10, + MaxRate: 100, + InitialRate: 50, + BackoffFactor: 0.5, + RecoveryFactor: 1.2, + }, + wantErr: true, + }, + { + name: "Invalid RecoveryMemoryThreshold negative", + config: AdaptiveThrottlerConfig{ + SampleInterval: 100 * time.Millisecond, + MaxMemoryPercent: 80, + RecoveryMemoryThreshold: -5, + MinRate: 10, + MaxRate: 100, + InitialRate: 50, + BackoffFactor: 0.5, + RecoveryFactor: 1.2, + }, + wantErr: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := tt.config.validate() + if tt.wantErr { + if err == nil { + t.Errorf("Expected error but got none") + } + } else { + if err != nil { + t.Errorf("Expected no error but got: %v", err) + } + } + }) + } +} + +func TestDefaultAdaptiveThrottlerConfig(t *testing.T) { + config := DefaultAdaptiveThrottlerConfig() + if config == nil { + t.Fatal("Expected config to be not nil") + } + if config.InitialRate != 1000 { + t.Errorf("Expected InitialRate to be 1000, got %d", config.InitialRate) + } + if config.MaxMemoryPercent != 85.0 { + t.Errorf("Expected MaxMemoryPercent to be 85.0, got %f", config.MaxMemoryPercent) + } + if config.MaxCPUPercent != 80.0 { + t.Errorf("Expected MaxCPUPercent to be 80.0, got %f", config.MaxCPUPercent) + } + if config.RecoveryMemoryThreshold != 0 { + t.Errorf("Expected RecoveryMemoryThreshold to be 0 (auto-calculated), got %f", config.RecoveryMemoryThreshold) + } + if config.RecoveryCPUThreshold != 0 { + t.Errorf("Expected RecoveryCPUThreshold to be 0 (auto-calculated), got %f", config.RecoveryCPUThreshold) + } + if !config.EnableHysteresis { + t.Errorf("Expected EnableHysteresis to be true, got %v", config.EnableHysteresis) + } +} + +func TestNewAdaptiveThrottler(t *testing.T) { + at1, err := NewAdaptiveThrottler(nil) + if err != nil { + t.Fatalf("Expected no error with nil config, got: %v", err) + } + if at1 == nil { + t.Fatal("Expected non-nil throttler") + } + if at1.config.InitialRate != 1000 { + t.Errorf("Expected default InitialRate 1000, got %d", at1.config.InitialRate) + } + at1.Close() + + config := DefaultAdaptiveThrottlerConfig() + config.InitialRate = 500 + at2, err := NewAdaptiveThrottler(config) + if err != nil { + t.Fatalf("Expected no error with valid config, got: %v", err) + } + if at2.config.InitialRate != 500 { + t.Errorf("Expected InitialRate 500, got %d", at2.config.InitialRate) + } + at2.Close() + + invalidConfig := &AdaptiveThrottlerConfig{ + SampleInterval: 1 * time.Millisecond, + } + at3, err := NewAdaptiveThrottler(invalidConfig) + if err == nil { + at3.Close() + t.Fatal("Expected error with invalid config") + } +} + +// TestAdaptiveThrottler_AdjustRate_Logic tests rate adjustment algorithm with high/low resource usage +func TestAdaptiveThrottler_AdjustRate_Logic(t *testing.T) { + config := DefaultAdaptiveThrottlerConfig() + config.MinRate = 10 + config.MaxRate = 100 + config.InitialRate = 50 + config.BackoffFactor = 0.5 + config.RecoveryFactor = 2.0 + config.MaxCPUPercent = 50.0 + config.MaxMemoryPercent = 50.0 + config.RecoveryCPUThreshold = 40.0 + config.RecoveryMemoryThreshold = 40.0 + + at, mockMonitor := createThrottlerForRateTesting(config, float64(config.InitialRate)) + + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 60.0, + MemoryUsedPercent: 30.0, + }) + at.adjustRate() + assert.InDelta(t, 42.5, at.GetCurrentRate(), 0.01) + + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 40.0, + MemoryUsedPercent: 60.0, + }) + at.adjustRate() + assert.InDelta(t, 36.125, at.GetCurrentRate(), 0.01) + + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 10.0, + MemoryUsedPercent: 10.0, + }) + at.adjustRate() + assert.InDelta(t, 46.9625, at.GetCurrentRate(), 0.01) +} + +// TestAdaptiveThrottler_Hysteresis tests hysteresis behavior with enabled/disabled modes +func TestAdaptiveThrottler_Hysteresis(t *testing.T) { + config := DefaultAdaptiveThrottlerConfig() + config.MinRate = 10 + config.MaxRate = 100 + config.InitialRate = 50 + config.BackoffFactor = 0.8 + config.RecoveryFactor = 1.2 + config.MaxCPUPercent = 80.0 + config.MaxMemoryPercent = 85.0 + config.RecoveryCPUThreshold = 70.0 + config.RecoveryMemoryThreshold = 75.0 + + // Test with hysteresis enabled (default) + config.EnableHysteresis = true + at, mockMonitor := createThrottlerForRateTesting(config, float64(config.InitialRate)) + + // CPU at 75% (above recovery threshold) - should not increase with hysteresis + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 75.0, + MemoryUsedPercent: 40.0, + }) + at.adjustRate() + assert.InDelta(t, 50.0, at.GetCurrentRate(), 0.01) + + // CPU at 65% (below recovery threshold) - should increase + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 65.0, + MemoryUsedPercent: 40.0, + }) + at.adjustRate() + assert.InDelta(t, 53.0, at.GetCurrentRate(), 0.01) + + // Test with hysteresis disabled + config.EnableHysteresis = false + at2, _ := createThrottlerForRateTesting(config, 50.0) + + // CPU at 75% (below max threshold) - should increase immediately + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 75.0, + MemoryUsedPercent: 40.0, + }) + at2.adjustRate() + assert.InDelta(t, 53.0, at2.GetCurrentRate(), 0.01) // Should increase immediately +} + +// TestAdaptiveThrottler_Limits tests that rate stays within min/max bounds +func TestAdaptiveThrottler_Limits(t *testing.T) { + config := DefaultAdaptiveThrottlerConfig() + config.MinRate = 10 + config.MaxRate = 20 + config.InitialRate = 10 + config.RecoveryFactor = 10.0 + config.RecoveryCPUThreshold = 70.0 + config.RecoveryMemoryThreshold = 75.0 + + at, mockMonitor := createThrottlerForRateTesting(config, 10.0) + + mockMonitor.ExpectGetStats(ResourceStats{CPUUsagePercent: 0, MemoryUsedPercent: 0}) + at.adjustRate() + assert.InDelta(t, 13.0, at.GetCurrentRate(), 0.01) +} + +// TestAdaptiveThrottler_FlowControl tests token bucket throttling with burst traffic +func TestAdaptiveThrottler_FlowControl(t *testing.T) { + mockMonitor := &MockMonitor{} + + config := DefaultAdaptiveThrottlerConfig() + config.InitialRate = 10 + config.SampleInterval = 10 * time.Second + + at := &AdaptiveThrottler{ + config: *config, + monitor: mockMonitor, + currentRateBits: math.Float64bits(float64(config.InitialRate)), + in: make(chan any), + out: make(chan any, 100), + done: make(chan struct{}), + } + + go at.monitorLoop() + go at.pipelineLoop() + + // Send events continuously for a period + sendDuration := 500 * time.Millisecond + sendDone := make(chan struct{}) + + sentCount := 0 + go func() { + defer close(sendDone) + ticker := time.NewTicker(1 * time.Millisecond) // Send very frequently + defer ticker.Stop() + + sendStart := time.Now() + for { + select { + case at.in <- sentCount: + sentCount++ + case <-ticker.C: + if time.Since(sendStart) >= sendDuration { + close(at.in) + return + } + } + } + }() + + // Collect received events + receivedCount := 0 + receiveDone := make(chan struct{}) + + go func() { + defer close(receiveDone) + for range at.out { + receivedCount++ + } + }() + + // Wait for sending to complete + <-sendDone + + // Wait a bit more for processing + time.Sleep(100 * time.Millisecond) + at.Close() + + // Wait for receiving to complete + <-receiveDone + + t.Logf("Sent %d events over %v, received %d events", sentCount, sendDuration, receivedCount) + + // With rate of 10/sec over 500ms, should receive about 5 events + expectedMax := int(float64(config.InitialRate) * sendDuration.Seconds() * 1.5) + if receivedCount > expectedMax { + t.Fatalf("Received too many events: got %d, expected at most %d", receivedCount, expectedMax) + } + if receivedCount < 1 { + t.Fatalf("Expected at least 1 event, got %d", receivedCount) + } +} + +// TestAdaptiveThrottler_To tests To method that streams data to a sink +func TestAdaptiveThrottler_To(t *testing.T) { + at := createThrottlerWithLongInterval(t) + defer at.Close() + + var received []any + var mu sync.Mutex + done := make(chan struct{}) + + sinkCh := make(chan any, 10) + mockSink := &mockSink{ + in: sinkCh, + completion: make(chan struct{}), + } + + // Collect data from sink + go collectDataFromChannelWithMutex(sinkCh, &received, &mu, done) + + testData := []any{"test1", "test2", "test3"} + + // Start the throttler streaming to sink + go at.To(mockSink) + + // Send test data + go func() { + for _, data := range testData { + at.In() <- data + } + close(at.In()) + }() + + // Wait for completion signal from sink + mockSink.AwaitCompletion() + + // Wait for receiver to finish + <-done + + mu.Lock() + defer mu.Unlock() + if len(received) != len(testData) { + t.Errorf("Expected %d items, got %d", len(testData), len(received)) + } + for i, expected := range testData { + if i >= len(received) || received[i] != expected { + t.Errorf("Expected data[%d] = %v, got %v", i, expected, received[i]) + } + } +} + +func TestAdaptiveThrottler_StreamPortioned(t *testing.T) { + mockMonitor := &MockMonitor{} + config := DefaultAdaptiveThrottlerConfig() + + at := &AdaptiveThrottler{ + config: *config, + monitor: mockMonitor, + currentRateBits: math.Float64bits(1000), + out: make(chan any, 10), + } + + inletIn := make(chan any, 10) + mockInlet := &mockInlet{in: inletIn} + + testData := []any{"data1", "data2", "data3"} + var received []any + var mu sync.Mutex + done := make(chan struct{}) + + // Start streamPortioned in background + go at.streamPortioned(mockInlet) + + // Collect all received data + go collectDataFromChannelWithMutex(inletIn, &received, &mu, done) + + // Send test data + go func() { + for _, data := range testData { + at.out <- data + } + close(at.out) + }() + + // Wait for all data to be received + <-done + + mu.Lock() + defer mu.Unlock() + if len(received) != len(testData) { + t.Errorf("Expected %d items, got %d", len(testData), len(received)) + } + for i, expected := range testData { + if received[i] != expected { + t.Errorf("Expected data[%d] = %v, got %v", i, expected, received[i]) + } + } +} + +func TestAdaptiveThrottler_Via_DataFlow(t *testing.T) { + // Actually test data flow + at, _ := NewAdaptiveThrottler(DefaultAdaptiveThrottlerConfig()) + defer at.Close() + + // Send test data and verify it flows through + testData := []any{"test1", "test2"} + go func() { + for _, data := range testData { + at.In() <- data + } + close(at.In()) + }() + + received := make([]any, 0, len(testData)) + for data := range at.Out() { + received = append(received, data) + } + + if len(received) != len(testData) { + t.Errorf("Expected %d items, got %d", len(testData), len(received)) + } +} + +// TestAdaptiveThrottler_AdjustRate_EdgeCases tests edge cases for adjustRate +func TestAdaptiveThrottler_AdjustRate_EdgeCases(t *testing.T) { + config := DefaultAdaptiveThrottlerConfig() + config.MinRate = 10 + config.MaxRate = 100 + config.InitialRate = 50 + config.BackoffFactor = 0.7 + config.RecoveryFactor = 1.3 + config.MaxCPUPercent = 80.0 + config.MaxMemoryPercent = 85.0 + config.RecoveryCPUThreshold = 70.0 + config.RecoveryMemoryThreshold = 75.0 + + at, mockMonitor := createThrottlerForRateTesting(config, 50.0) + + // Both CPU and memory constrained + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 90.0, + MemoryUsedPercent: 90.0, + }) + at.adjustRate() + assert.InDelta(t, 45.5, at.GetCurrentRate(), 0.1) + + // Rate at max, should not exceed + at.setRate(100.0) + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 10.0, + MemoryUsedPercent: 10.0, + }) + at.adjustRate() + rate := at.GetCurrentRate() + if rate > 100.0 { + t.Errorf("Rate should not exceed MaxRate, got %f", rate) + } + + // Rate at min, constrained + at.setRate(10.0) + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 90.0, + MemoryUsedPercent: 90.0, + }) + at.adjustRate() + rate = at.GetCurrentRate() + if rate < 0 { + t.Errorf("Rate should not be negative, got %f", rate) + } + + // Exactly at thresholds + at.setRate(50.0) + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 80.0, // Exactly at max + MemoryUsedPercent: 70.0, + }) + at.adjustRate() + rate = at.GetCurrentRate() + if rate < 0 { + t.Errorf("Rate should not be negative, got %f", rate) + } + + // Exactly at recovery thresholds with hysteresis + at.setRate(50.0) + config.EnableHysteresis = true + mockMonitor.ExpectGetStats(ResourceStats{ + CPUUsagePercent: 70.0, // Exactly at recovery threshold + MemoryUsedPercent: 75.0, + }) + at.adjustRate() + rate = at.GetCurrentRate() + if rate > 50.0 { + t.Errorf("With hysteresis, rate should not increase at threshold, got %f", rate) + } +} + +// TestAdaptiveThrottler_PipelineLoop_Shutdown tests pipelineLoop shutdown scenarios +func TestAdaptiveThrottler_PipelineLoop_Shutdown(t *testing.T) { + mockMonitor := &MockMonitor{} + config := DefaultAdaptiveThrottlerConfig() + config.InitialRate = 100 // Fast rate for testing + + at := &AdaptiveThrottler{ + config: *config, + monitor: mockMonitor, + currentRateBits: math.Float64bits(100.0), + in: make(chan any), + out: make(chan any, 10), + done: make(chan struct{}), + } + + go at.pipelineLoop() + + at.in <- "test1" + time.Sleep(10 * time.Millisecond) + + close(at.done) + time.Sleep(200 * time.Millisecond) + + select { + case _, ok := <-at.out: + if ok { + time.Sleep(100 * time.Millisecond) + select { + case _, ok2 := <-at.out: + if ok2 { + t.Error("Output channel should be closed after shutdown") + } + default: + } + } + default: + } +} + +// TestAdaptiveThrottler_PipelineLoop_RateChange tests pipelineLoop with rate changes +func TestAdaptiveThrottler_PipelineLoop_RateChange(t *testing.T) { + mockMonitor := &MockMonitor{} + config := DefaultAdaptiveThrottlerConfig() + + at := &AdaptiveThrottler{ + config: *config, + monitor: mockMonitor, + currentRateBits: math.Float64bits(10.0), // Start at 10/sec + in: make(chan any), + out: make(chan any, 10), + done: make(chan struct{}), + } + + go at.pipelineLoop() + + // Send items and change rate during processing + go func() { + for i := 0; i < 5; i++ { + at.in <- i + time.Sleep(10 * time.Millisecond) + // Change rate mid-stream + if i == 2 { + at.setRate(20.0) // Double the rate + } + } + close(at.in) + }() + + // Collect items + received := make([]any, 0) + timeout := time.After(2 * time.Second) + for { + select { + case item, ok := <-at.out: + if !ok { + goto done + } + received = append(received, item) + case <-timeout: + t.Fatal("Timeout waiting for items") + } + } +done: + + if len(received) != 5 { + t.Errorf("Expected 5 items, got %d", len(received)) + } + + // Close done to clean up + close(at.done) +} + +// TestAdaptiveThrottler_PipelineLoop_LowRate tests pipelineLoop with very low rate +func TestAdaptiveThrottler_PipelineLoop_LowRate(t *testing.T) { + mockMonitor := &MockMonitor{} + config := DefaultAdaptiveThrottlerConfig() + + at := &AdaptiveThrottler{ + config: *config, + monitor: mockMonitor, + currentRateBits: math.Float64bits(0.5), // Very slow: 0.5/sec + in: make(chan any), + out: make(chan any, 10), + done: make(chan struct{}), + } + + go at.pipelineLoop() + + // Send one item + at.in <- "test" + close(at.in) + + // Should receive it (rate < 1.0 is clamped to 1.0) + select { + case item := <-at.out: + if item != "test" { + t.Errorf("Expected 'test', got %v", item) + } + case <-time.After(2 * time.Second): + t.Fatal("Timeout waiting for item") + } + + // Wait for channel to close + select { + case _, ok := <-at.out: + if ok { + t.Error("Output channel should be closed") + } + case <-time.After(100 * time.Millisecond): + // Channel should be closed by now + } + + close(at.done) +} + +// TestAdaptiveThrottler_To_Shutdown tests To method with shutdown +func TestAdaptiveThrottler_To_Shutdown(t *testing.T) { + at := createThrottlerWithLongInterval(t) + + mockSink := newMockSinkWithChannelDrain() + sinkCh := mockSink.in + + // Start To in background + toDone := make(chan struct{}) + go func() { + defer close(toDone) + at.To(mockSink) + }() + + // Send some data + go func() { + for i := 0; i < 3; i++ { + at.In() <- i + } + close(at.In()) + }() + + // Wait a bit for data to start flowing + time.Sleep(100 * time.Millisecond) + + at.Close() + + // Wait for To to complete (it will finish when streamPortioned completes) + select { + case <-toDone: + case <-time.After(2 * time.Second): + t.Error("To method did not complete within timeout") + } + + time.Sleep(100 * time.Millisecond) + + verifyChannelClosed(t, sinkCh, 50*time.Millisecond) +} + +// TestAdaptiveThrottler_StreamPortioned_Blocking tests streamPortioned with blocking inlet +func TestAdaptiveThrottler_StreamPortioned_Blocking(t *testing.T) { + mockMonitor := &MockMonitor{} + config := DefaultAdaptiveThrottlerConfig() + + at := &AdaptiveThrottler{ + config: *config, + monitor: mockMonitor, + currentRateBits: math.Float64bits(1000), + out: make(chan any), + } + + // Unbuffered inlet to test blocking behavior + inletIn := make(chan any) + mockInlet := &mockInlet{in: inletIn} + + // Start streamPortioned in background + done := make(chan struct{}) + go func() { + defer close(done) + at.streamPortioned(mockInlet) + }() + + // Send data + go func() { + at.out <- "test1" + at.out <- "test2" + close(at.out) + }() + + // Read from inlet (unblocking the sender) + received := collectDataFromChannel(inletIn) + + // Wait for completion + select { + case <-done: + // Good + case <-time.After(1 * time.Second): + t.Fatal("Timeout waiting for streamPortioned to complete") + } + + // Verify inlet is closed + verifyChannelClosed(t, inletIn, 50*time.Millisecond) + + if len(received) != 2 { + t.Errorf("Expected 2 items, got %d", len(received)) + } +} + +func TestAdaptiveThrottler_RealisticProductionScenarios(t *testing.T) { + tests := []struct { + name string + config func() *AdaptiveThrottlerConfig + scenario string + steps []ResourceStats + }{ + { + name: "High CPU Production Scenario", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 80.0 + config.MaxMemoryPercent = 90.0 + config.InitialRate = 1000 + config.MinRate = 50 + config.MaxRate = 5000 + config.BackoffFactor = 0.7 + config.RecoveryFactor = 1.2 + config.RecoveryCPUThreshold = 70.0 + config.RecoveryMemoryThreshold = 80.0 + return config + }, + scenario: "High CPU usage scenario typical in production", + steps: []ResourceStats{ + {CPUUsagePercent: 85.0, MemoryUsedPercent: 60.0}, + {CPUUsagePercent: 75.0, MemoryUsedPercent: 65.0}, + {CPUUsagePercent: 65.0, MemoryUsedPercent: 70.0}, + {CPUUsagePercent: 55.0, MemoryUsedPercent: 75.0}, + }, + }, + { + name: "Memory Pressure Production Scenario", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 90.0 + config.MaxMemoryPercent = 85.0 + config.InitialRate = 2000 + config.MinRate = 100 + config.MaxRate = 10000 + config.BackoffFactor = 0.6 + config.RecoveryFactor = 1.3 + config.RecoveryCPUThreshold = 80.0 + config.RecoveryMemoryThreshold = 75.0 + return config + }, + scenario: "Memory pressure scenario common in memory-intensive apps", + steps: []ResourceStats{ + {CPUUsagePercent: 70.0, MemoryUsedPercent: 88.0}, + {CPUUsagePercent: 75.0, MemoryUsedPercent: 82.0}, + {CPUUsagePercent: 65.0, MemoryUsedPercent: 78.0}, + {CPUUsagePercent: 60.0, MemoryUsedPercent: 72.0}, + }, + }, + { + name: "Balanced Production Load", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 75.0 + config.MaxMemoryPercent = 80.0 + config.InitialRate = 1500 + config.MinRate = 200 + config.MaxRate = 8000 + config.BackoffFactor = 0.75 + config.RecoveryFactor = 1.25 + config.RecoveryCPUThreshold = 65.0 + config.RecoveryMemoryThreshold = 70.0 + return config + }, + scenario: "Balanced load typical for well-tuned production systems", + steps: []ResourceStats{ + {CPUUsagePercent: 78.0, MemoryUsedPercent: 65.0}, + {CPUUsagePercent: 72.0, MemoryUsedPercent: 68.0}, + {CPUUsagePercent: 68.0, MemoryUsedPercent: 75.0}, + {CPUUsagePercent: 65.0, MemoryUsedPercent: 72.0}, + }, + }, + { + name: "Conservative Production Settings", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 60.0 + config.MaxMemoryPercent = 75.0 + config.InitialRate = 500 + config.MinRate = 50 + config.MaxRate = 2000 + config.BackoffFactor = 0.8 + config.RecoveryFactor = 1.1 + config.RecoveryCPUThreshold = 50.0 + config.RecoveryMemoryThreshold = 65.0 + return config + }, + scenario: "Conservative settings for critical production systems", + steps: []ResourceStats{ + {CPUUsagePercent: 65.0, MemoryUsedPercent: 70.0}, + {CPUUsagePercent: 58.0, MemoryUsedPercent: 72.0}, + {CPUUsagePercent: 55.0, MemoryUsedPercent: 68.0}, + {CPUUsagePercent: 52.0, MemoryUsedPercent: 65.0}, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + config := tt.config() + at, mockMonitor := createThrottlerForRateTesting(config, float64(config.InitialRate)) + + t.Logf("Testing scenario: %s", tt.scenario) + + throttlingTriggered := false + + for i, stats := range tt.steps { + initialRate := at.GetCurrentRate() + mockMonitor.ExpectGetStats(stats) + at.adjustRate() + finalRate := at.GetCurrentRate() + + isConstrained := stats.CPUUsagePercent > config.MaxCPUPercent || + stats.MemoryUsedPercent > config.MaxMemoryPercent + + isBelowRecovery := stats.CPUUsagePercent < config.RecoveryCPUThreshold && + stats.MemoryUsedPercent < config.RecoveryMemoryThreshold + + t.Logf("Step %d: CPU %.1f%%, Mem %.1f%% -> Rate %.1f (constrained: %v, below recovery: %v)", + i+1, stats.CPUUsagePercent, stats.MemoryUsedPercent, finalRate, isConstrained, isBelowRecovery) + + // Verify throttling behavior + if isConstrained { + throttlingTriggered = true + if finalRate > initialRate { + t.Errorf("Step %d: Rate should not increase when constrained, but %.1f > %.1f", + i+1, finalRate, initialRate) + } + } + + // Verify recovery behavior (only if we've seen throttling before) + if throttlingTriggered && !isConstrained { + if config.EnableHysteresis && !isBelowRecovery { + // With hysteresis, rate should not increase until both resources are below recovery thresholds + if finalRate > initialRate { + t.Errorf("Step %d: With hysteresis, rate should not increase until both resources below recovery thresholds", + i+1) + } + } else if !config.EnableHysteresis || isBelowRecovery { + // Without hysteresis or when below recovery thresholds, rate should be able to increase + if finalRate < initialRate { + t.Errorf("Step %d: Rate should not decrease during recovery, but %.1f < %.1f", + i+1, finalRate, initialRate) + } + } + } + + // Verify rate bounds + if finalRate < float64(config.MinRate) { + t.Errorf("Step %d: Rate %.1f below MinRate %d", i+1, finalRate, config.MinRate) + } + if finalRate > float64(config.MaxRate) { + t.Errorf("Step %d: Rate %.1f above MaxRate %d", i+1, finalRate, config.MaxRate) + } + } + + // Ensure we actually tested throttling behavior + if !throttlingTriggered { + t.Errorf("Test scenario should have triggered throttling at least once") + } + }) + } +} + +// TestAdaptiveThrottler_SustainedLoadScenarios tests behavior under prolonged load +func TestAdaptiveThrottler_SustainedLoadScenarios(t *testing.T) { + tests := []struct { + name string + config func() *AdaptiveThrottlerConfig + loadPattern []ResourceStats + description string + }{ + { + name: "Sustained High CPU Load", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 75.0 + config.MaxMemoryPercent = 90.0 + config.InitialRate = 2000 + config.MinRate = 100 + config.MaxRate = 10000 + config.BackoffFactor = 0.7 + config.RecoveryFactor = 1.2 + return config + }, + loadPattern: []ResourceStats{ + {CPUUsagePercent: 80.0, MemoryUsedPercent: 60.0}, + {CPUUsagePercent: 82.0, MemoryUsedPercent: 62.0}, + {CPUUsagePercent: 78.0, MemoryUsedPercent: 64.0}, + {CPUUsagePercent: 85.0, MemoryUsedPercent: 66.0}, + {CPUUsagePercent: 81.0, MemoryUsedPercent: 68.0}, + }, + description: "Sustained high CPU usage with some variation", + }, + { + name: "Memory Pressure Buildup", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 85.0 + config.MaxMemoryPercent = 80.0 + config.InitialRate = 3000 + config.MinRate = 200 + config.MaxRate = 15000 + config.BackoffFactor = 0.6 + config.RecoveryFactor = 1.15 + return config + }, + loadPattern: []ResourceStats{ + {CPUUsagePercent: 70.0, MemoryUsedPercent: 75.0}, + {CPUUsagePercent: 72.0, MemoryUsedPercent: 78.0}, + {CPUUsagePercent: 68.0, MemoryUsedPercent: 82.0}, + {CPUUsagePercent: 71.0, MemoryUsedPercent: 85.0}, + {CPUUsagePercent: 69.0, MemoryUsedPercent: 83.0}, + }, + description: "Gradual memory pressure buildup typical of memory leaks", + }, + { + name: "Mixed Resource Contention", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 70.0 + config.MaxMemoryPercent = 75.0 + config.InitialRate = 2500 + config.MinRate = 300 + config.MaxRate = 12000 + config.BackoffFactor = 0.65 + config.RecoveryFactor = 1.25 + config.EnableHysteresis = true + return config + }, + loadPattern: []ResourceStats{ + {CPUUsagePercent: 75.0, MemoryUsedPercent: 72.0}, + {CPUUsagePercent: 68.0, MemoryUsedPercent: 78.0}, + {CPUUsagePercent: 72.0, MemoryUsedPercent: 76.0}, + {CPUUsagePercent: 69.0, MemoryUsedPercent: 74.0}, + {CPUUsagePercent: 66.0, MemoryUsedPercent: 71.0}, + }, + description: "Mixed CPU and memory pressure with hysteresis", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + config := tt.config() + at, mockMonitor := createThrottlerForRateTesting(config, float64(config.InitialRate)) + + t.Logf("Testing sustained load: %s", tt.description) + + var finalRate float64 + for i, stats := range tt.loadPattern { + mockMonitor.ExpectGetStats(stats) + at.adjustRate() + finalRate = at.GetCurrentRate() + + t.Logf("Iteration %d: CPU %.1f%%, Mem %.1f%% -> Rate %.1f", + i+1, stats.CPUUsagePercent, stats.MemoryUsedPercent, finalRate) + } + + // Calculate expected rate range algorithmically + minExpectedRate, maxExpectedRate := calculateExpectedRateRange(config, float64(config.InitialRate), tt.loadPattern) + + t.Logf("Expected final rate range: [%.1f, %.1f], actual: %.1f", minExpectedRate, maxExpectedRate, finalRate) + + if finalRate < minExpectedRate || finalRate > maxExpectedRate { + t.Errorf("Final rate %.1f outside expected range [%.1f, %.1f] for sustained load scenario", + finalRate, minExpectedRate, maxExpectedRate) + } + + // Verify rate doesn't oscillate wildly in final iterations + // (This would be a sign of poor hysteresis or smoothing) + if finalRate < float64(config.MinRate)*0.9 { + t.Errorf("Final rate %.1f too close to MinRate %d, indicating possible oscillation", + finalRate, config.MinRate) + } + }) + } +} + +// TestAdaptiveThrottler_RecoveryScenarios tests recovery from high load to normal load +func TestAdaptiveThrottler_RecoveryScenarios(t *testing.T) { + tests := []struct { + name string + config func() *AdaptiveThrottlerConfig + highLoad ResourceStats + normalLoad ResourceStats + description string + }{ + { + name: "CPU Spike Recovery", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 80.0 + config.MaxMemoryPercent = 90.0 + config.InitialRate = 2000 + config.MinRate = 200 + config.MaxRate = 10000 + config.BackoffFactor = 0.6 + config.RecoveryFactor = 1.4 + config.SampleInterval = 100 * time.Millisecond + config.RecoveryCPUThreshold = 70.0 + config.RecoveryMemoryThreshold = 80.0 + return config + }, + highLoad: ResourceStats{CPUUsagePercent: 85.0, MemoryUsedPercent: 70.0}, + normalLoad: ResourceStats{CPUUsagePercent: 60.0, MemoryUsedPercent: 65.0}, + description: "Recovery from CPU spike to normal load", + }, + { + name: "Memory Pressure Recovery", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 90.0 + config.MaxMemoryPercent = 75.0 + config.InitialRate = 3000 + config.MinRate = 300 + config.MaxRate = 15000 + config.BackoffFactor = 0.5 + config.RecoveryFactor = 1.3 + config.RecoveryCPUThreshold = 80.0 + config.RecoveryMemoryThreshold = 65.0 + return config + }, + highLoad: ResourceStats{CPUUsagePercent: 70.0, MemoryUsedPercent: 85.0}, + normalLoad: ResourceStats{CPUUsagePercent: 65.0, MemoryUsedPercent: 60.0}, + description: "Recovery from memory pressure to normal load", + }, + { + name: "Dual Resource Recovery with Hysteresis", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 75.0 + config.MaxMemoryPercent = 80.0 + config.InitialRate = 2500 + config.MinRate = 250 + config.MaxRate = 12000 + config.BackoffFactor = 0.65 + config.RecoveryFactor = 1.25 + config.EnableHysteresis = true + config.RecoveryCPUThreshold = 65.0 + config.RecoveryMemoryThreshold = 70.0 + return config + }, + highLoad: ResourceStats{CPUUsagePercent: 85.0, MemoryUsedPercent: 85.0}, + normalLoad: ResourceStats{CPUUsagePercent: 60.0, MemoryUsedPercent: 65.0}, + description: "Recovery from dual resource pressure with hysteresis", + }, + { + name: "Recovery Without Hysteresis", + config: func() *AdaptiveThrottlerConfig { + config := DefaultAdaptiveThrottlerConfig() + config.MaxCPUPercent = 75.0 + config.MaxMemoryPercent = 80.0 + config.InitialRate = 2500 + config.MinRate = 250 + config.MaxRate = 12000 + config.BackoffFactor = 0.65 + config.RecoveryFactor = 1.25 + config.EnableHysteresis = false + config.RecoveryCPUThreshold = 65.0 + config.RecoveryMemoryThreshold = 70.0 + return config + }, + highLoad: ResourceStats{CPUUsagePercent: 85.0, MemoryUsedPercent: 85.0}, + normalLoad: ResourceStats{CPUUsagePercent: 70.0, MemoryUsedPercent: 75.0}, + description: "Recovery without hysteresis (faster recovery)", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + config := tt.config() + at, mockMonitor := createThrottlerForRateTesting(config, float64(config.InitialRate)) + + t.Logf("Testing recovery: %s", tt.description) + t.Logf("Config - CPU Max: %.1f%% (Recovery: %.1f%%), Mem Max: %.1f%% (Recovery: %.1f%%), Hysteresis: %v", + config.MaxCPUPercent, config.RecoveryCPUThreshold, + config.MaxMemoryPercent, config.RecoveryMemoryThreshold, config.EnableHysteresis) + + // Start with high load - should trigger throttling + mockMonitor.ExpectGetStats(tt.highLoad) + at.adjustRate() + highLoadRate := at.GetCurrentRate() + + if highLoadRate >= float64(config.InitialRate) { + t.Errorf("High load should reduce rate, but got %.1f >= %d", + highLoadRate, config.InitialRate) + } + + t.Logf("High load (CPU:%.1f%%, Mem:%.1f%%) -> Rate: %.1f", + tt.highLoad.CPUUsagePercent, tt.highLoad.MemoryUsedPercent, highLoadRate) + + // Transition to normal load - recovery behavior depends on hysteresis + mockMonitor.ExpectGetStats(tt.normalLoad) + at.adjustRate() + recoveryRate1 := at.GetCurrentRate() + + t.Logf("Normal load (CPU:%.1f%%, Mem:%.1f%%) -> Rate: %.1f", + tt.normalLoad.CPUUsagePercent, tt.normalLoad.MemoryUsedPercent, recoveryRate1) + + // With hysteresis disabled, rate should increase immediately if not constrained + isConstrained := tt.normalLoad.CPUUsagePercent > config.MaxCPUPercent || + tt.normalLoad.MemoryUsedPercent > config.MaxMemoryPercent + + if !isConstrained && !config.EnableHysteresis { + if recoveryRate1 <= highLoadRate { + t.Errorf("Without hysteresis, rate should increase when not constrained, but %.1f <= %.1f", + recoveryRate1, highLoadRate) + } + } + + // Continue recovery for a few more cycles to allow hysteresis recovery + for i := 0; i < 5; i++ { + mockMonitor.ExpectGetStats(tt.normalLoad) + at.adjustRate() + } + finalRecoveryRate := at.GetCurrentRate() + + t.Logf("After recovery cycles: Rate: %.1f", finalRecoveryRate) + + // Calculate expected recovery rate algorithmically + expectedRecoveryRate := calculateExpectedRecoveryRate( + config, + float64(config.InitialRate), + tt.highLoad, + tt.normalLoad, + 5, + ) + tolerance := 0.05 // 5% tolerance for floating point precision + + t.Logf("Expected recovery rate: %.1f, actual: %.1f", expectedRecoveryRate, finalRecoveryRate) + + // Verify recovery rate is within expected range + minExpected := expectedRecoveryRate * (1 - tolerance) + maxExpected := expectedRecoveryRate * (1 + tolerance) + + if finalRecoveryRate < minExpected || finalRecoveryRate > maxExpected { + t.Errorf("Final recovery rate %.1f outside expected range [%.1f, %.1f]", + finalRecoveryRate, minExpected, maxExpected) + } + + // Should not exceed MaxRate + if finalRecoveryRate > float64(config.MaxRate) { + t.Errorf("Recovery rate %.1f should not exceed MaxRate %d", + finalRecoveryRate, config.MaxRate) + } + }) + } +} diff --git a/flow/util.go b/flow/operators.go similarity index 100% rename from flow/util.go rename to flow/operators.go diff --git a/flow/resource_monitor.go b/flow/resource_monitor.go new file mode 100644 index 0000000..cb47ede --- /dev/null +++ b/flow/resource_monitor.go @@ -0,0 +1,386 @@ +package flow + +import ( + "fmt" + "log/slog" + "math" + "runtime" + "sync" + "sync/atomic" + "time" + + "github.com/reugn/go-streams/internal/sysmonitor" +) + +// CPUUsageMode defines strategies for CPU usage monitoring. +type CPUUsageMode int + +const ( + // CPUUsageModeHeuristic uses goroutine count as a lightweight CPU usage proxy. + CPUUsageModeHeuristic CPUUsageMode = iota + // CPUUsageModeMeasured provides accurate CPU usage measurement via system calls. + CPUUsageModeMeasured +) + +// ResourceStats holds current system resource utilization metrics. +type ResourceStats struct { + MemoryUsedPercent float64 // Memory usage as a percentage (0-100). + CPUUsagePercent float64 // CPU usage as a percentage (0-100). + GoroutineCount int // Number of active goroutines. + Timestamp time.Time // Time when these stats were collected. +} + +// resourceMonitor defines the interface for resource monitoring. +type resourceMonitor interface { + GetStats() ResourceStats + Close() +} + +// ResourceMonitor collects and provides system resource usage statistics. +type ResourceMonitor struct { + sampleInterval time.Duration + mu sync.Mutex // Protects configuration changes. + cpuMode CPUUsageMode // Current CPU monitoring strategy. + memoryReader func() (float64, error) // Custom memory usage reader. + + // Runtime state + stats atomic.Pointer[ResourceStats] // Latest resource statistics. + sampler sysmonitor.ProcessCPUSampler // CPU usage sampler implementation. + memoryReaderInstance sysmonitor.ProcessMemoryReader // System memory reader instance. + updateIntervalCh chan time.Duration // Channel for dynamic interval updates. + done chan struct{} // Signals monitoring loop termination. + closeOnce sync.Once // Ensures clean shutdown. +} + +// newResourceMonitor creates a new resource monitor instance. +// This constructor is private and should only be called by the registry. +func newResourceMonitor( + sampleInterval time.Duration, + cpuMode CPUUsageMode, + memoryReader func() (float64, error), +) *ResourceMonitor { + rm := &ResourceMonitor{ + sampleInterval: sampleInterval, + cpuMode: cpuMode, + memoryReader: memoryReader, + updateIntervalCh: make(chan time.Duration, 1), + done: make(chan struct{}), + } + + // Initialize with empty stats + rm.stats.Store(&ResourceStats{ + Timestamp: time.Now(), + }) + + rm.initSampler() + rm.initMemoryReader() + + go rm.monitor() + return rm +} + +// GetStats returns the most recent resource usage statistics. +// The returned data is thread-safe and represents a consistent snapshot. +func (rm *ResourceMonitor) GetStats() ResourceStats { + val := rm.stats.Load() + if val == nil { + return ResourceStats{} + } + return *val +} + +// GetMode returns the current CPU monitoring strategy. +func (rm *ResourceMonitor) GetMode() CPUUsageMode { + rm.mu.Lock() + defer rm.mu.Unlock() + return rm.cpuMode +} + +// SetMode changes the CPU monitoring strategy if possible. +// Allows switching between heuristic and measured modes. +func (rm *ResourceMonitor) SetMode(newMode CPUUsageMode) { + rm.mu.Lock() + defer rm.mu.Unlock() + + // No change needed + if newMode == rm.cpuMode { + return + } + + switch newMode { + case CPUUsageModeMeasured: + // Try to switch to measured mode + if sampler, err := sysmonitor.NewCPUSampler(sysmonitor.OSFileSystem{}); err == nil { + rm.sampler = sampler + rm.cpuMode = CPUUsageModeMeasured + } else { + slog.Error("failed to switch to measured mode", "error", err) + } + case CPUUsageModeHeuristic: + rm.sampler = sysmonitor.NewGoroutineHeuristicSampler() + rm.cpuMode = CPUUsageModeHeuristic + } +} + +// initSampler initializes the appropriate CPU usage sampler. +// Uses measured mode by default if available +func (rm *ResourceMonitor) initSampler() { + if sampler, err := sysmonitor.NewCPUSampler(sysmonitor.OSFileSystem{}); err == nil { + rm.sampler = sampler + rm.cpuMode = CPUUsageModeMeasured + } else { + // Fallback to heuristic + rm.sampler = sysmonitor.NewGoroutineHeuristicSampler() + rm.cpuMode = CPUUsageModeHeuristic + } +} + +// initMemoryReader initializes the system memory reader. +// This reader is reused across all sampling operations. +func (rm *ResourceMonitor) initMemoryReader() { + rm.memoryReaderInstance = sysmonitor.NewProcessMemoryReader(sysmonitor.OSFileSystem{}) +} + +// monitor runs the continuous resource sampling loop. +// Handles dynamic interval changes and graceful shutdown. +func (rm *ResourceMonitor) monitor() { + ticker := time.NewTicker(rm.sampleInterval) + defer ticker.Stop() + + for { + select { + case <-rm.done: + return + case d := <-rm.updateIntervalCh: + rm.mu.Lock() + if d != rm.sampleInterval { + rm.sampleInterval = d + ticker.Stop() + ticker = time.NewTicker(rm.sampleInterval) + rm.sample() + } + rm.mu.Unlock() + case <-ticker.C: + rm.sample() + } + } +} + +// sample collects current CPU, memory, and goroutine statistics. +// Updates the atomic stats pointer with the latest measurements. +func (rm *ResourceMonitor) sample() { + stats := &ResourceStats{ + Timestamp: time.Now(), + GoroutineCount: runtime.NumGoroutine(), + } + + // Memory Usage + switch { + // Check if a custom memory reader is provided + case rm.memoryReader != nil: + if mem, err := rm.memoryReader(); err == nil { + stats.MemoryUsedPercent = mem + } + case rm.memoryReaderInstance != nil: + if memStats, err := rm.memoryReaderInstance.Sample(); err == nil && memStats.Total > 0 { + used := memStats.Total - memStats.Available + stats.MemoryUsedPercent = float64(used) / float64(memStats.Total) * 100 + } + default: + // Fallback to runtime memory stats + var m runtime.MemStats + runtime.ReadMemStats(&m) + if m.Sys > 0 { + stats.MemoryUsedPercent = float64(m.Alloc) / float64(m.Sys) * 100 + } + } + + // CPU Usage + stats.CPUUsagePercent = rm.sampler.Sample(rm.sampleInterval) + + rm.stats.Store(stats) +} + +// getSampleInterval thread-safe getter of the current sample interval. +func (rm *ResourceMonitor) getSampleInterval() time.Duration { + rm.mu.Lock() + defer rm.mu.Unlock() + return rm.sampleInterval +} + +// setInterval updates the sampling frequency dynamically. +func (rm *ResourceMonitor) setInterval(d time.Duration) { + select { + case rm.updateIntervalCh <- d: + case <-rm.done: + } +} + +// stop terminates the monitoring goroutine gracefully. +func (rm *ResourceMonitor) stop() { + rm.closeOnce.Do(func() { + close(rm.done) + }) +} + +// globalMonitorRegistry manages the singleton ResourceMonitor instance. +// Provides shared access with reference counting and automatic cleanup. +var globalMonitorRegistry = &monitorRegistry{ + intervalRefs: make(map[time.Duration]int), +} + +// monitorIdleTimeout defines how long to wait before stopping an unused monitor. +// Prevents unnecessary recreation of the monitor instance when new throttler is added. +const monitorIdleTimeout = 5 * time.Second + +// monitorRegistry coordinates shared access to a ResourceMonitor instance. +// Manages multiple consumers with different sampling requirements efficiently. +type monitorRegistry struct { + mu sync.Mutex // Protects registry state. + instance *ResourceMonitor // The shared monitor instance. + intervalRefs map[time.Duration]int // Reference counts per sampling interval. + currentMin time.Duration // Current minimum sampling interval. + stopTimer *time.Timer // Timer for delayed cleanup. +} + +// Acquire obtains a handle to the shared resource monitor. +// Manages reference counting and may create or reconfigure the monitor as needed. +func (r *monitorRegistry) Acquire( + requestedInterval time.Duration, + cpuMode CPUUsageMode, + memReader func() (float64, error), +) resourceMonitor { + r.mu.Lock() + defer r.mu.Unlock() + + // Validate the requested interval + if requestedInterval <= 0 { + panic(fmt.Sprintf("resource monitor: invalid interval %v, must be positive", requestedInterval)) + } + + // Cancel pending stop if we are resurrecting within the grace period + if r.stopTimer != nil { + if !r.stopTimer.Stop() { + <-r.stopTimer.C + } + r.stopTimer = nil + } + + // Register the requested interval + r.intervalRefs[requestedInterval]++ + + // Calculate global minimum + requiredMin := r.calculateMinInterval() + + if r.instance == nil { + r.instance = newResourceMonitor(requiredMin, cpuMode, memReader) + r.currentMin = requiredMin + } else { + // Check if we need to upgrade the existing instance + if cpuMode > r.instance.GetMode() { + r.instance.SetMode(cpuMode) + } + + // Adjust interval if this new user needs it faster + if requiredMin != r.currentMin { + r.instance.setInterval(requiredMin) + r.currentMin = requiredMin + } + } + + return &sharedMonitorHandle{ + monitor: r.instance, + interval: requestedInterval, + registry: r, + } +} + +// release decrements the reference count for a sampling interval. +// Initiates cleanup when no consumers remain. +func (r *monitorRegistry) release(interval time.Duration) { + r.mu.Lock() + defer r.mu.Unlock() + + r.intervalRefs[interval]-- + if r.intervalRefs[interval] <= 0 { + delete(r.intervalRefs, interval) + } + + if len(r.intervalRefs) == 0 { + // Start grace period timer + if r.stopTimer == nil { + r.stopTimer = time.AfterFunc(monitorIdleTimeout, func() { + r.cleanup() + }) + } + return + } + + // If we still have users, check if we can slow down (release pressure) + newMin := r.calculateMinInterval() + if newMin != r.currentMin && r.instance != nil { + r.instance.setInterval(newMin) + r.currentMin = newMin + } +} + +// cleanup stops and destroys the monitor instance. +// Called after the idle timeout when no consumers remain. +func (r *monitorRegistry) cleanup() { + r.mu.Lock() + defer r.mu.Unlock() + + // Double check we are still empty + if len(r.intervalRefs) == 0 && r.instance != nil { + r.instance.stop() + r.instance = nil + } + r.stopTimer = nil +} + +// calculateMinInterval finds the fastest sampling rate required by any consumer. +// Returns a default interval when no consumers are registered. +func (r *monitorRegistry) calculateMinInterval() time.Duration { + if len(r.intervalRefs) == 0 { + return time.Second + } + minInterval := time.Duration(math.MaxInt64) + for d := range r.intervalRefs { + if d < minInterval { + minInterval = d + } + } + return minInterval +} + +// getInstanceSampleInterval returns the current instance's sample interval in a thread-safe manner. +// Returns 0 if no instance exists. +func (r *monitorRegistry) getInstanceSampleInterval() time.Duration { + r.mu.Lock() + defer r.mu.Unlock() + if r.instance == nil { + return 0 + } + return r.instance.getSampleInterval() +} + +// sharedMonitorHandle provides consumers with access to the shared monitor. +// Ensures proper reference counting and cleanup when no longer needed. +type sharedMonitorHandle struct { + monitor *ResourceMonitor // Reference to the shared monitor. + interval time.Duration // Sampling interval requested by this consumer. + registry *monitorRegistry // Registry managing this handle. + once sync.Once // Ensures Close is called only once. +} + +// GetStats returns resource statistics from the shared monitor. +func (h *sharedMonitorHandle) GetStats() ResourceStats { + return h.monitor.GetStats() +} + +// Close releases this consumer's reference to the shared monitor. +func (h *sharedMonitorHandle) Close() { + h.once.Do(func() { + h.registry.release(h.interval) + }) +} diff --git a/flow/resource_monitor_test.go b/flow/resource_monitor_test.go new file mode 100644 index 0000000..1fc105a --- /dev/null +++ b/flow/resource_monitor_test.go @@ -0,0 +1,940 @@ +package flow + +import ( + "fmt" + "math" + "sync" + "testing" + "time" + + "github.com/reugn/go-streams/internal/sysmonitor" +) + +const ( + testSampleInterval = 10 * time.Millisecond + testStopTimeout = 50 * time.Millisecond + testStatsUpdateMargin = 20 * time.Millisecond +) + +type mockCPUSampler struct { + val float64 +} + +func (m *mockCPUSampler) Sample(_ time.Duration) float64 { + return m.val +} + +func (m *mockCPUSampler) Reset() { + // No-op for mock +} + +func (m *mockCPUSampler) IsInitialized() bool { + return true // Mock is always initialized +} + +type assertError string + +func (e assertError) Error() string { return string(e) } + +func resetRegistry() { + globalMonitorRegistry = &monitorRegistry{ + intervalRefs: make(map[time.Duration]int), + } +} + +func setupTest(t *testing.T) { + t.Helper() + resetRegistry() + t.Cleanup(resetRegistry) +} + +// assertValidStats checks that resource stats contain reasonable values +func assertValidStats(t *testing.T, stats ResourceStats) { + t.Helper() + if stats.Timestamp.IsZero() { + t.Error("Timestamp should not be zero") + } + if stats.GoroutineCount < 0 { + t.Errorf("GoroutineCount should not be negative, got %d", stats.GoroutineCount) + } + if stats.MemoryUsedPercent < 0 || stats.MemoryUsedPercent > 200 { // Allow >100% for some systems + t.Errorf("MemoryUsedPercent should be between 0-200, got %f", stats.MemoryUsedPercent) + } + if stats.CPUUsagePercent < 0 { + t.Errorf("CPUUsagePercent should not be negative, got %f", stats.CPUUsagePercent) + } +} + +// TestResourceMonitor_Initialization tests monitor creation with different CPU modes +func TestResourceMonitor_Initialization(t *testing.T) { + setupTest(t) + + // Test with Heuristic preference - should use best available (measured if possible) + rm := newResourceMonitor(time.Second, CPUUsageModeHeuristic, nil) + if rm.sampler == nil { + t.Error("Expected sampler to be initialized, got nil") + } + // Should use measured mode if available, regardless of preference + if rm.cpuMode != CPUUsageModeMeasured && rm.cpuMode != CPUUsageModeHeuristic { + t.Errorf("Unexpected CPU mode: %v", rm.cpuMode) + } + rm.stop() + + // Test with Measured preference - should use measured if available + rm2 := newResourceMonitor(time.Second, CPUUsageModeMeasured, nil) + if rm2.sampler == nil { + t.Error("Expected sampler to be initialized, got nil") + } + // Should use measured mode if available + if rm2.cpuMode != CPUUsageModeMeasured && rm2.cpuMode != CPUUsageModeHeuristic { + t.Errorf("Unexpected CPU mode: %v", rm2.cpuMode) + } + rm2.stop() +} + +// TestResourceMonitor_Sample_MemoryReader tests memory sampling with custom reader +func TestResourceMonitor_Sample_MemoryReader(t *testing.T) { + setupTest(t) + + expectedMem := 42.5 + mockMemReader := func() (float64, error) { + return expectedMem, nil + } + + // Create monitor with long interval to prevent auto-sampling during test + rm := newResourceMonitor(time.Hour, CPUUsageModeHeuristic, mockMemReader) + defer rm.stop() + + // Get initial stats before manual sample + initialStats := rm.GetStats() + + // Trigger sample manually + rm.sample() + + stats := rm.GetStats() + + // Verify Memory + if stats.MemoryUsedPercent != expectedMem { + t.Errorf("Expected memory %f, got %f", expectedMem, stats.MemoryUsedPercent) + } + + // Verify stats are valid and updated + assertValidStats(t, stats) + if !stats.Timestamp.After(initialStats.Timestamp) { + t.Error("Timestamp should be updated after sampling") + } +} + +// TestResourceMonitor_Sample_CPUMock tests CPU sampling with mock sampler +func TestResourceMonitor_Sample_CPUMock(t *testing.T) { + setupTest(t) + + expectedCPU := 12.34 + mockSampler := &mockCPUSampler{val: expectedCPU} + + // Initialize with Measured mode + rm := newResourceMonitor(time.Hour, CPUUsageModeMeasured, nil) + defer rm.stop() + + // Inject mock sampler manually (overwriting the real one) + rm.sampler = mockSampler + + rm.sample() + stats := rm.GetStats() + + if stats.CPUUsagePercent != expectedCPU { + t.Errorf("Expected CPU %f, got %f", expectedCPU, stats.CPUUsagePercent) + } +} + +// TestResourceMonitor_Sample_HeuristicMode tests heuristic CPU sampling +func TestResourceMonitor_Sample_HeuristicMode(t *testing.T) { + setupTest(t) + + rm := newResourceMonitor(time.Hour, CPUUsageModeHeuristic, nil) + defer rm.stop() + + // Force heuristic mode by setting the sampler directly + rm.sampler = sysmonitor.NewGoroutineHeuristicSampler() + rm.cpuMode = CPUUsageModeHeuristic + + rm.sample() + stats := rm.GetStats() + + // Heuristic mode should use the GoroutineHeuristicSampler + // The sampler returns a sophisticated calculation based on goroutine count + if stats.CPUUsagePercent <= 0 { + t.Errorf("Expected positive CPU usage in heuristic mode, got %f", stats.CPUUsagePercent) + } + if stats.CPUUsagePercent > 100 { + t.Errorf("Expected CPU usage <= 100%%, got %f", stats.CPUUsagePercent) + } +} + +// TestMonitorRegistry_Acquire tests registry acquire logic with different intervals +func TestMonitorRegistry_Acquire(t *testing.T) { + setupTest(t) + + // 1. First Acquire + h1 := globalMonitorRegistry.Acquire(2*time.Second, CPUUsageModeHeuristic, nil) + defer h1.Close() + + if globalMonitorRegistry.instance == nil { + t.Fatal("Registry instance should not be nil after acquire") + } + if globalMonitorRegistry.currentMin != 2*time.Second { + t.Errorf("Expected currentMin 2s, got %v", globalMonitorRegistry.currentMin) + } + if count := globalMonitorRegistry.intervalRefs[2*time.Second]; count != 1 { + t.Errorf("Expected ref count 1, got %d", count) + } + + // 2. Second Acquire (Same interval) - should just increment ref + h2 := globalMonitorRegistry.Acquire(2*time.Second, CPUUsageModeHeuristic, nil) + defer h2.Close() + + if count := globalMonitorRegistry.intervalRefs[2*time.Second]; count != 2 { + t.Errorf("Expected ref count 2, got %d", count) + } + + // 3. Third Acquire (Faster interval) - should update monitor + h3 := globalMonitorRegistry.Acquire(1*time.Second, CPUUsageModeHeuristic, nil) + defer h3.Close() + + if globalMonitorRegistry.currentMin != 1*time.Second { + t.Errorf("Expected currentMin to update to 1s, got %v", globalMonitorRegistry.currentMin) + } + + // Wait for the monitor goroutine to process the interval update (buffered channel makes this async) + timeout := time.After(100 * time.Millisecond) + for { + select { + case <-timeout: + t.Fatalf("Timeout waiting for instance interval update. Expected 1s, got %v", + globalMonitorRegistry.getInstanceSampleInterval()) + default: + if globalMonitorRegistry.getInstanceSampleInterval() == 1*time.Second { + goto intervalUpdated + } + time.Sleep(1 * time.Millisecond) + } + } +intervalUpdated: + + // 4. Fourth Acquire (Slower interval) - should NOT update monitor + h4 := globalMonitorRegistry.Acquire(5*time.Second, CPUUsageModeHeuristic, nil) + defer h4.Close() + + if globalMonitorRegistry.currentMin != 1*time.Second { + t.Errorf("Expected currentMin to remain 1s, got %v", globalMonitorRegistry.currentMin) + } +} + +// TestMonitorRegistry_Release_Logic tests registry release and cleanup logic +func TestMonitorRegistry_Release_Logic(t *testing.T) { + setupTest(t) + + // Acquire 1s and 5s + hFast := globalMonitorRegistry.Acquire(1*time.Second, CPUUsageModeHeuristic, nil) + hSlow := globalMonitorRegistry.Acquire(5*time.Second, CPUUsageModeHeuristic, nil) + + // Initial State + if globalMonitorRegistry.currentMin != 1*time.Second { + t.Fatalf("Setup failed: expected 1s min") + } + + // Release Fast Handle + hFast.Close() + + // Registry should check if it can slow down + // Now only 5s is left, so min should become 5s + if globalMonitorRegistry.currentMin != 5*time.Second { + t.Errorf("Expected currentMin to relax to 5s, got %v", globalMonitorRegistry.currentMin) + } + + // Release Slow Handle + hSlow.Close() + + // Check that ref count is zero + if len(globalMonitorRegistry.intervalRefs) != 0 { + t.Errorf("Expected empty refs, got %v", globalMonitorRegistry.intervalRefs) + } + + // Verify cleanup timer is started + if globalMonitorRegistry.stopTimer == nil { + t.Error("Expected stopTimer to be set after full release") + } + + // Force cleanup manually to verify cleanup logic works (instead of waiting 5s) + globalMonitorRegistry.cleanup() + if globalMonitorRegistry.instance != nil { + t.Error("Expected instance to be nil after cleanup") + } +} + +// TestMonitorRegistry_Resurrect tests registry resurrection after cleanup timer starts +func TestMonitorRegistry_Resurrect(t *testing.T) { + setupTest(t) + + // Acquire and Release to trigger timer + h1 := globalMonitorRegistry.Acquire(time.Second, CPUUsageModeHeuristic, nil) + h1.Close() + + if globalMonitorRegistry.stopTimer == nil { + t.Fatal("Timer should be running") + } + + // Acquire again BEFORE timer fires + h2 := globalMonitorRegistry.Acquire(time.Second, CPUUsageModeHeuristic, nil) + defer h2.Close() + + if globalMonitorRegistry.stopTimer != nil { + t.Error("Timer should be stopped/nil after resurrection") + } + if globalMonitorRegistry.instance == nil { + t.Error("Instance should still be alive") + } +} + +// TestResourceMonitor_SetInterval_Dynamic tests dynamic interval changes +func TestResourceMonitor_SetInterval_Dynamic(t *testing.T) { + setupTest(t) + + rm := newResourceMonitor(10*time.Minute, CPUUsageModeHeuristic, nil) + defer rm.stop() + + // Since we can't easily hook into the loop, we observe the effect. + // We change interval to something very short, which should trigger 'sample()' + + // Current state + initialStats := rm.GetStats() + + // Change interval + newDuration := time.Millisecond + rm.setInterval(newDuration) + + // Wait for update + // The loop: case d := <-rm.updateIntervalCh -> sets rm.sampleInterval -> calls rm.sample() + + // We poll briefly for the change + deadline := time.Now().Add(1 * time.Second) + updated := false + for time.Now().Before(deadline) { + if rm.getSampleInterval() == newDuration { + updated = true + break + } + time.Sleep(10 * time.Millisecond) + } + + if !updated { + t.Error("Failed to update interval dynamically") + } + + // Ensure stats timestamp updated (proof that sample() was called on switch) + time.Sleep(time.Millisecond) + currentStats := rm.GetStats() + if !currentStats.Timestamp.After(initialStats.Timestamp) { + t.Error("Expected immediate sample on interval switch") + } +} + +// TestSharedMonitorHandle tests handle functionality and idempotent close +func TestSharedMonitorHandle(t *testing.T) { + setupTest(t) + + h := globalMonitorRegistry.Acquire(time.Second, CPUUsageModeHeuristic, nil) + + // Test GetStats delegates correctly + stats := h.GetStats() + if stats.Timestamp.IsZero() { + t.Error("Handle returned zero stats") + } + + // Test Idempotent Close + h.Close() + // Access internal registry to verify release happened + if len(globalMonitorRegistry.intervalRefs) != 0 { + t.Error("Registry not empty after close") + } + + // Second close should not panic and not change refs (already 0) + func() { + defer func() { + if r := recover(); r != nil { + t.Errorf("Double close panicked: %v", r) + } + }() + h.Close() + }() +} + +// TestMemoryFallback tests fallback to runtime memory stats when reader is nil +func TestMemoryFallback(t *testing.T) { + setupTest(t) + + // Test fallback when memoryReader is nil + rm := newResourceMonitor(time.Hour, CPUUsageModeHeuristic, nil) + defer rm.stop() + + rm.sample() + stats := rm.GetStats() + + // Memory percentage should be a valid value (0-100) + if stats.MemoryUsedPercent < 0 || stats.MemoryUsedPercent > 100 { + t.Errorf("Expected memory percentage in range [0,100], got %f", stats.MemoryUsedPercent) + } + + // Should have other stats populated + if stats.GoroutineCount <= 0 { + t.Errorf("Expected goroutine count > 0, got %d", stats.GoroutineCount) + } + if stats.CPUUsagePercent < 0 { + t.Errorf("Expected CPU usage >= 0, got %f", stats.CPUUsagePercent) + } + if stats.Timestamp.IsZero() { + t.Error("Expected non-zero timestamp") + } +} + +// TestMemoryReaderError tests error handling in memory reader with fallback +func TestMemoryReaderError(t *testing.T) { + setupTest(t) + + // Reader that returns error + errReader := func() (float64, error) { + return 0, assertError("fail") + } + + rm := newResourceMonitor(time.Hour, CPUUsageModeHeuristic, errReader) + defer rm.stop() + + initialStats := rm.GetStats() + rm.sample() + stats := rm.GetStats() + + // Should fall back to runtime stats + if stats.MemoryUsedPercent < 0 { + t.Errorf("Expected memory percentage >= 0, got %f", stats.MemoryUsedPercent) + } + if stats.MemoryUsedPercent > 100 { + t.Errorf("Expected memory percentage <= 100, got %f", stats.MemoryUsedPercent) + } + + // Other stats should still be valid + if stats.GoroutineCount <= 0 { + t.Errorf("Expected goroutine count > 0, got %d", stats.GoroutineCount) + } + if stats.CPUUsagePercent < 0 { + t.Errorf("Expected CPU usage >= 0, got %f", stats.CPUUsagePercent) + } + if !stats.Timestamp.After(initialStats.Timestamp) { + t.Error("Expected timestamp to be updated") + } +} + +// TestResourceMonitor_CPU_SamplerError tests CPU sampler error handling +func TestResourceMonitor_CPU_SamplerError(t *testing.T) { + setupTest(t) + + // Create monitor and inject a failing sampler + rm := newResourceMonitor(time.Hour, CPUUsageModeMeasured, nil) + defer rm.stop() + + // Create a mock sampler that returns negative values (error condition) + failingSampler := &mockCPUSampler{val: -1.0} + rm.sampler = failingSampler + + rm.sample() + stats := rm.GetStats() + + // CPU sampler can return negative values, so we just check it's a valid float + // The implementation doesn't clamp negative values, it passes them through + if math.IsNaN(stats.CPUUsagePercent) { + t.Error("Expected valid CPU usage value, got NaN") + } + + // Other stats should still be valid + if stats.GoroutineCount <= 0 { + t.Errorf("Expected goroutine count > 0, got %d", stats.GoroutineCount) + } + if stats.MemoryUsedPercent < 0 { + t.Errorf("Expected memory percentage >= 0, got %f", stats.MemoryUsedPercent) + } +} + +// TestResourceMonitor_SetMode tests CPU monitoring mode switching +func TestResourceMonitor_SetMode(t *testing.T) { + setupTest(t) + + // Create monitor - should try measured mode first + rm := newResourceMonitor(time.Second, CPUUsageModeHeuristic, nil) + defer rm.stop() + + // Should use measured mode if available (new behavior) + initialMode := rm.GetMode() + if initialMode != CPUUsageModeMeasured && initialMode != CPUUsageModeHeuristic { + t.Errorf("Unexpected initial mode: %v", initialMode) + } + + // Test switching to Heuristic mode + rm.SetMode(CPUUsageModeHeuristic) + if rm.GetMode() != CPUUsageModeHeuristic { + t.Errorf("Expected mode Heuristic after switching, got %v", rm.GetMode()) + } + + // Test switching back to Measured mode + rm.SetMode(CPUUsageModeMeasured) + if rm.GetMode() != CPUUsageModeMeasured { + t.Errorf("Expected mode Measured after switching back, got %v", rm.GetMode()) + } + + // Test switching to the same mode (should be no-op) + rm.SetMode(CPUUsageModeMeasured) + if rm.GetMode() != CPUUsageModeMeasured { + t.Errorf("Expected mode to remain Measured, got %v", rm.GetMode()) + } +} + +// TestMonitorRegistry_BufferedChannel_DeadlockPrevention +func TestMonitorRegistry_BufferedChannel_DeadlockPrevention(t *testing.T) { + setupTest(t) + + // Create a blocking memory reader to simulate slow I/O + sampleStarted := make(chan bool, 1) + sampleContinue := make(chan bool, 1) + blockingMemoryReader := func() (float64, error) { + sampleStarted <- true + <-sampleContinue // Block until signaled to continue + return 50.0, nil + } + + // Start first acquire with blocking memory reader + handle1 := globalMonitorRegistry.Acquire(200*time.Millisecond, CPUUsageModeHeuristic, blockingMemoryReader) + defer handle1.Close() + + // Wait for monitor to start sampling and block + select { + case <-sampleStarted: + // Monitor is now blocked in memoryReader + case <-time.After(1 * time.Second): + t.Fatal("Monitor didn't start sampling within timeout") + } + + // Now try to acquire with different interval - this triggers setInterval + // With unbuffered channel, this would deadlock holding registry lock + start := time.Now() + handle2 := globalMonitorRegistry.Acquire(50*time.Millisecond, CPUUsageModeHeuristic, nil) + defer handle2.Close() + + // The acquire should complete quickly with buffered channel + elapsed := time.Since(start) + if elapsed > 100*time.Millisecond { + t.Errorf("Acquire took too long (%v) - possible blocking", elapsed) + } + + // Unblock the memory reader so monitor can process the update + sampleContinue <- true + + // Give monitor time to process the update + time.Sleep(50 * time.Millisecond) +} + +// TestMonitorRegistry_ConcurrentAccess tests concurrent access to the registry +func TestMonitorRegistry_ConcurrentAccess(t *testing.T) { + setupTest(t) + + const numGoroutines = 5 + const operationsPerGoroutine = 20 + + var wg sync.WaitGroup + errors := make(chan error, numGoroutines*operationsPerGoroutine) + + // Launch multiple goroutines performing concurrent operations + for i := 0; i < numGoroutines; i++ { + wg.Add(1) + go func(id int) { + defer wg.Done() + for j := 0; j < operationsPerGoroutine; j++ { + switch j % 3 { + case 0: // Acquire and release + handle := globalMonitorRegistry.Acquire(time.Duration(id+1)*time.Millisecond, CPUUsageModeHeuristic, nil) + handle.Close() + case 1: // Get stats if available + if globalMonitorRegistry.instance != nil { + _ = globalMonitorRegistry.instance.GetStats() + } + case 2: // Registry inspection + globalMonitorRegistry.mu.Lock() + _ = len(globalMonitorRegistry.intervalRefs) + globalMonitorRegistry.mu.Unlock() + } + } + }(i) + } + + wg.Wait() + close(errors) + + // Check for any errors (panics would be caught here) + for err := range errors { + t.Errorf("Concurrent access error: %v", err) + } +} + +// TestResourceMonitor_BoundaryConditions tests edge cases and boundary conditions +func TestResourceMonitor_BoundaryConditions(t *testing.T) { + tests := []struct { + name string + interval time.Duration + expectedValid bool + }{ + {"very small interval", time.Nanosecond, true}, + {"very large interval", 24 * time.Hour, true}, + {"normal interval", time.Second, true}, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + setupTest(t) + + // Test registry acquire with boundary intervals + handle := globalMonitorRegistry.Acquire(tt.interval, CPUUsageModeHeuristic, nil) + defer handle.Close() + + // Should not panic and should return valid stats + stats := handle.GetStats() + if stats.Timestamp.IsZero() { + t.Error("Expected valid timestamp") + } + + // For valid intervals, check that monitor was created + if tt.expectedValid { + if globalMonitorRegistry.instance == nil { + t.Error("Expected monitor instance to be created") + } + } + }) + } + + // Test invalid intervals cause validation panic + t.Run("invalid intervals", func(t *testing.T) { + testCases := []struct { + name string + interval time.Duration + wantMsg string + }{ + {"negative interval", -time.Second, "resource monitor: invalid interval -1s, must be positive"}, + {"zero interval", 0, "resource monitor: invalid interval 0s, must be positive"}, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + setupTest(t) + + defer func() { + if r := recover(); r == nil { + t.Errorf("Expected panic for %s", tc.name) + } else if msg := fmt.Sprintf("%v", r); msg != tc.wantMsg { + t.Errorf("Expected panic message %q, got %q", tc.wantMsg, msg) + } + }() + + // This should panic with validation error + globalMonitorRegistry.Acquire(tc.interval, CPUUsageModeHeuristic, nil) + }) + } + }) +} + +// TestResourceMonitor_Stop tests the stop method for cleanup and shutdown +func TestResourceMonitor_Stop(t *testing.T) { + setupTest(t) + + rm := newResourceMonitor(testSampleInterval, CPUUsageModeHeuristic, nil) + + // Verify monitor is running + initialStats := rm.GetStats() + if initialStats.Timestamp.IsZero() { + t.Fatal("Monitor should be running and producing stats") + } + + // Wait for sampling to occur + time.Sleep(testStopTimeout) + statsBeforeStop := rm.GetStats() + if !statsBeforeStop.Timestamp.After(initialStats.Timestamp) { + t.Fatal("Monitor should have sampled at least once") + } + + // Stop the monitor + rm.stop() + + // Verify monitor has stopped by checking no more updates occur + time.Sleep(testStopTimeout) + statsAfterStop := rm.GetStats() + + // Allow small timing variations but ensure no significant updates + if statsAfterStop.Timestamp.Sub(statsBeforeStop.Timestamp) > testStatsUpdateMargin { + t.Error("Monitor should have stopped, stats should not update significantly") + } + + // Test idempotent stop + rm.stop() // Should not panic + rm.stop() // Should not panic +} + +// TestResourceMonitor_Stop_Concurrent tests concurrent stop calls +func TestResourceMonitor_Stop_Concurrent(t *testing.T) { + setupTest(t) + + rm := newResourceMonitor(100*time.Millisecond, CPUUsageModeHeuristic, nil) + + // Call stop concurrently + var wg sync.WaitGroup + for i := 0; i < 10; i++ { + wg.Add(1) + go func() { + defer wg.Done() + rm.stop() + }() + } + + wg.Wait() + + // Should not panic and monitor should be stopped + time.Sleep(200 * time.Millisecond) + stats1 := rm.GetStats() + time.Sleep(200 * time.Millisecond) + stats2 := rm.GetStats() + + // Stats should not update after stop + if stats2.Timestamp.After(stats1.Timestamp.Add(50 * time.Millisecond)) { + t.Error("Monitor should be stopped, stats should not update") + } +} + +// TestResourceMonitor_GetStats_Concurrent tests concurrent GetStats calls +func TestResourceMonitor_GetStats_Concurrent(t *testing.T) { + setupTest(t) + + rm := newResourceMonitor(100*time.Millisecond, CPUUsageModeHeuristic, nil) + defer rm.stop() + + // Call GetStats concurrently + var wg sync.WaitGroup + errors := make(chan error, 100) + + for i := 0; i < 50; i++ { + wg.Add(1) + go func() { + defer wg.Done() + stats := rm.GetStats() + // Verify stats are valid + if stats.Timestamp.IsZero() { + errors <- fmt.Errorf("got zero timestamp") + } + if stats.GoroutineCount < 0 { + errors <- fmt.Errorf("got invalid goroutine count: %d", stats.GoroutineCount) + } + }() + } + + wg.Wait() + close(errors) + + // Check for errors + for err := range errors { + t.Error(err) + } +} + +// TestSharedMonitorHandle_Close tests Close method for cleanup +func TestSharedMonitorHandle_Close(t *testing.T) { + setupTest(t) + + // Acquire a handle + h1 := globalMonitorRegistry.Acquire(time.Second, CPUUsageModeHeuristic, nil) + + // Verify registry has reference + if len(globalMonitorRegistry.intervalRefs) == 0 { + t.Error("Registry should have reference after acquire") + } + + // Close the handle + h1.Close() + + // Verify reference is released + if len(globalMonitorRegistry.intervalRefs) != 0 { + t.Error("Registry should have no references after close") + } + + // Test idempotent close + h1.Close() // Should not panic + h1.Close() // Should not panic + + // Verify still no references + if len(globalMonitorRegistry.intervalRefs) != 0 { + t.Error("Registry should still have no references after multiple closes") + } +} + +// TestSharedMonitorHandle_Close_MultipleHandles tests Close with multiple handles +func TestSharedMonitorHandle_Close_MultipleHandles(t *testing.T) { + setupTest(t) + + // Acquire multiple handles with same interval + h1 := globalMonitorRegistry.Acquire(time.Second, CPUUsageModeHeuristic, nil) + h2 := globalMonitorRegistry.Acquire(time.Second, CPUUsageModeHeuristic, nil) + h3 := globalMonitorRegistry.Acquire(time.Second, CPUUsageModeHeuristic, nil) + + // Verify ref count is 3 + if count := globalMonitorRegistry.intervalRefs[time.Second]; count != 3 { + t.Errorf("Expected ref count 3, got %d", count) + } + + // Close one handle + h1.Close() + + // Verify ref count is 2 + if count := globalMonitorRegistry.intervalRefs[time.Second]; count != 2 { + t.Errorf("Expected ref count 2, got %d", count) + } + + // Close remaining handles + h2.Close() + h3.Close() + + // Verify no references + if len(globalMonitorRegistry.intervalRefs) != 0 { + t.Error("Registry should have no references after all closes") + } +} + +// TestSharedMonitorHandle_Close_Concurrent tests concurrent Close calls +func TestSharedMonitorHandle_Close_Concurrent(t *testing.T) { + setupTest(t) + + // Acquire multiple handles + handles := make([]resourceMonitor, 10) + for i := 0; i < 10; i++ { + handles[i] = globalMonitorRegistry.Acquire(time.Second, CPUUsageModeHeuristic, nil) + } + + // Close all handles concurrently + var wg sync.WaitGroup + for _, h := range handles { + wg.Add(1) + go func(handle resourceMonitor) { + defer wg.Done() + handle.Close() + }(h) + } + + wg.Wait() + + // Verify all references are released + if len(globalMonitorRegistry.intervalRefs) != 0 { + t.Error("Registry should have no references after concurrent closes") + } +} + +// TestMonitorRegistry_CalculateMinInterval tests calculateMinInterval function +func TestMonitorRegistry_CalculateMinInterval(t *testing.T) { + tests := []struct { + name string + intervals map[time.Duration]int + expected time.Duration + }{ + {"empty registry", nil, time.Second}, + {"single interval", map[time.Duration]int{2 * time.Second: 1}, 2 * time.Second}, + {"multiple intervals", map[time.Duration]int{ + 5 * time.Second: 1, + 1 * time.Second: 1, + 3 * time.Second: 1, + }, 1 * time.Second}, + {"very small interval", map[time.Duration]int{50 * time.Millisecond: 1}, 50 * time.Millisecond}, + {"very large interval", map[time.Duration]int{24 * time.Hour: 1}, 24 * time.Hour}, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + setupTest(t) // Ensure clean state + + // Set up test intervals + globalMonitorRegistry.intervalRefs = tt.intervals + + minInterval := globalMonitorRegistry.calculateMinInterval() + if minInterval != tt.expected { + t.Errorf("Expected %v, got %v", tt.expected, minInterval) + } + }) + } +} + +// TestResourceMonitor_GetStats_NilStats tests GetStats when stats pointer is nil +func TestResourceMonitor_GetStats_NilStats(t *testing.T) { + setupTest(t) + + rm := newResourceMonitor(time.Hour, CPUUsageModeHeuristic, nil) + defer rm.stop() + + // Manually set stats to nil to simulate uninitialized state + rm.stats.Store(nil) + + // GetStats should return empty ResourceStats without panicking + stats := rm.GetStats() + + // Verify it returns zero/empty stats + if !stats.Timestamp.IsZero() { + t.Error("Expected zero timestamp for nil stats") + } + if stats.CPUUsagePercent != 0 { + t.Errorf("Expected zero CPU usage, got %f", stats.CPUUsagePercent) + } + if stats.MemoryUsedPercent != 0 { + t.Errorf("Expected zero memory usage, got %f", stats.MemoryUsedPercent) + } + if stats.GoroutineCount != 0 { + t.Errorf("Expected zero goroutine count, got %d", stats.GoroutineCount) + } +} + +// TestResourceMonitor_Sample_RuntimeMemoryFallback tests sample method fallback to runtime memory stats +func TestResourceMonitor_Sample_RuntimeMemoryFallback(t *testing.T) { + setupTest(t) + + // Create monitor with nil memory reader and instance to force fallback + rm := newResourceMonitor(time.Hour, CPUUsageModeHeuristic, nil) + defer rm.stop() + + // Ensure memory reader instance is nil (fallback path) + rm.memoryReaderInstance = nil + + // Record initial stats before sampling + initialStats := rm.GetStats() + + // Trigger sampling + rm.sample() + + // Get updated stats + stats := rm.GetStats() + + // Verify basic stats are populated + assertValidStats(t, stats) + if !stats.Timestamp.After(initialStats.Timestamp) { + t.Error("Timestamp should be updated after sampling") + } + + // Verify CPU usage is set (even if 0, it's still assigned) + // Note: CPU usage could be 0 in heuristic mode, which is valid + + // Verify goroutine count is reasonable + if stats.GoroutineCount <= 0 { + t.Errorf("Expected goroutine count > 0, got %d", stats.GoroutineCount) + } + + // Memory usage should be a valid percentage (runtime.ReadMemStats fallback) + if stats.MemoryUsedPercent < 0 || stats.MemoryUsedPercent > 100 { + t.Errorf("Expected memory percentage in range [0,100], got %f", stats.MemoryUsedPercent) + } +} diff --git a/go.mod b/go.mod index 6bead2d..de9d75c 100644 --- a/go.mod +++ b/go.mod @@ -1,3 +1,3 @@ module github.com/reugn/go-streams -go 1.21 +go 1.21.0 diff --git a/internal/assert/assertions.go b/internal/assert/assertions.go index f656b0a..75494f8 100644 --- a/internal/assert/assertions.go +++ b/internal/assert/assertions.go @@ -1,6 +1,7 @@ package assert import ( + "math" "reflect" "strings" "testing" @@ -43,3 +44,11 @@ func Panics(t *testing.T, f func()) { }() f() } + +// InDelta checks whether two float64 values are within a given delta of each other. +func InDelta(t *testing.T, expected, actual, delta float64) { + if math.Abs(expected-actual) > delta { + t.Helper() + t.Fatalf("Expected %v to be within %v of %v", actual, delta, expected) + } +} diff --git a/internal/sysmonitor/cpu.go b/internal/sysmonitor/cpu.go new file mode 100644 index 0000000..eeda870 --- /dev/null +++ b/internal/sysmonitor/cpu.go @@ -0,0 +1,36 @@ +package sysmonitor + +import ( + "time" +) + +// ProcessCPUSampler provides a cross-platform interface for sampling +// CPU usage of the current process. Returns normalized CPU usage (0-100%) +// across all available CPU cores. +// +// Platform implementations: +// - Linux: reads from /proc/[pid]/stat +// - Darwin (macOS): uses syscall.Getrusage +// - Windows: uses GetProcessTimes API +// - Other platforms: returns error +type ProcessCPUSampler interface { + // Sample returns normalized CPU usage percentage (0-100%) since last sample. + // On first call, initializes state and returns 0.0. If elapsed time since + // last sample is less than half of deltaTime, returns last known value. + // Returns last known value on error. + Sample(deltaTime time.Duration) float64 + + // Reset clears sampler state for a new session. Next Sample call will + // behave as the first sample. + Reset() + + // IsInitialized returns true if at least one sample has been taken. + IsInitialized() bool +} + +// NewProcessSampler creates a CPU sampler for the current process. +// Automatically selects platform-specific implementation (Linux/Darwin/Windows). +// Returns error on unsupported platforms or if sampler creation fails. +func NewCPUSampler(fs FileSystem) (ProcessCPUSampler, error) { + return newPlatformCPUSampler(fs) +} diff --git a/internal/sysmonitor/cpu_darwin.go b/internal/sysmonitor/cpu_darwin.go new file mode 100644 index 0000000..a35c7cd --- /dev/null +++ b/internal/sysmonitor/cpu_darwin.go @@ -0,0 +1,112 @@ +//go:build darwin + +package sysmonitor + +import ( + "fmt" + "math" + "os" + "runtime" + "syscall" + "time" +) + +// darwinProcessSampler samples CPU usage for the current process on macOS +type darwinProcessSampler struct { + pid int + lastUTime float64 + lastSTime float64 + lastSample time.Time + lastPercent float64 +} + +// newPlatformCPUSampler matches the factory signature required by cpu.go. +func newPlatformCPUSampler(_ FileSystem) (ProcessCPUSampler, error) { + pid := os.Getpid() + if pid < 0 || pid > math.MaxInt32 { + return nil, fmt.Errorf("invalid PID: %d", pid) + } + + return &darwinProcessSampler{ + pid: pid, + }, nil +} + +// Sample returns the CPU usage percentage since the last sample +func (s *darwinProcessSampler) Sample(deltaTime time.Duration) float64 { + utime, stime, err := s.readProcessTimesDarwin() + if err != nil { + return s.lastPercent + } + + now := time.Now() + if s.lastSample.IsZero() { + s.lastUTime = utime + s.lastSTime = stime + s.lastSample = now + s.lastPercent = 0.0 + return 0.0 + } + + elapsed := now.Sub(s.lastSample) + if elapsed < deltaTime/2 { + return s.lastPercent + } + + prevTotal := s.lastUTime + s.lastSTime + currTotal := utime + stime + cpuTimeDelta := currTotal - prevTotal + wallTimeSeconds := elapsed.Seconds() + + if wallTimeSeconds <= 0 { + return s.lastPercent + } + + // Normalized to 0-100% (divides by numCPU for system-wide metric) + numcpu := runtime.NumCPU() + if numcpu <= 0 { + numcpu = 1 // Safety check + } + percent := (cpuTimeDelta / wallTimeSeconds) * 100.0 / float64(numcpu) + + if percent < 0.0 { + percent = 0.0 + } else if percent > 100.0 { + percent = 100.0 + } + + s.lastUTime = utime + s.lastSTime = stime + s.lastSample = now + s.lastPercent = percent + + return percent +} + +// Reset clears sampler state for a new session +func (s *darwinProcessSampler) Reset() { + s.lastUTime = 0 + s.lastSTime = 0 + s.lastSample = time.Time{} + s.lastPercent = 0.0 +} + +// IsInitialized returns true if at least one sample has been taken +func (s *darwinProcessSampler) IsInitialized() bool { + return !s.lastSample.IsZero() +} + +// readProcessTimesDarwin reads CPU times via syscall.Getrusage (returns seconds) +func (s *darwinProcessSampler) readProcessTimesDarwin() (utime, stime float64, err error) { + var rusage syscall.Rusage + + err = syscall.Getrusage(syscall.RUSAGE_SELF, &rusage) + if err != nil { + return 0, 0, fmt.Errorf("failed to get process resource usage: %w", err) + } + + // Convert Timeval (Sec/Usec) to float64 seconds + utime = float64(rusage.Utime.Sec) + float64(rusage.Utime.Usec)/1e6 + stime = float64(rusage.Stime.Sec) + float64(rusage.Stime.Usec)/1e6 + return utime, stime, nil +} diff --git a/internal/sysmonitor/cpu_darwin_test.go b/internal/sysmonitor/cpu_darwin_test.go new file mode 100644 index 0000000..86af86d --- /dev/null +++ b/internal/sysmonitor/cpu_darwin_test.go @@ -0,0 +1,75 @@ +//go:build darwin + +package sysmonitor + +import ( + "testing" + "time" +) + +func TestDarwinSamplerIntegration(t *testing.T) { + sampler, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Darwin sampler: %v", err) + } + + if sampler.IsInitialized() { + t.Error("Should not be initialized initially") + } + + // First sample + sampler.Sample(time.Second) + if !sampler.IsInitialized() { + t.Error("Should be initialized after sample") + } + + // Generate CPU load + go func() { + // Burn CPU for 100ms + end := time.Now().Add(100 * time.Millisecond) + for time.Now().Before(end) { + } + }() + + // Wait and sample + time.Sleep(200 * time.Millisecond) + + val := sampler.Sample(0) + t.Logf("Darwin CPU Sample: %f%%", val) + + if val <= 0.0 { + t.Error("Expected detected CPU usage > 0") + } + + // Test bounds checking by creating edge case scenarios + testBoundsChecking(t) +} + +func testBoundsChecking(t *testing.T) { + samplerRaw, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Darwin sampler: %v", err) + } + sampler := samplerRaw.(*darwinProcessSampler) + + // First sample to initialize + sampler.Sample(time.Second) + + // Test early return when elapsed time is too short + // (this tests the "elapsed < deltaTime/2" condition) + shortInterval := 10 * time.Millisecond + result := sampler.Sample(shortInterval) + + // Should return the last known value (0.0) due to short interval + if result != 0.0 { + t.Errorf("Expected last value 0.0 for short interval, got %f", result) + } + + // Test that bounds checking works (though hard to trigger negative values in real usage) + // The bounds checking (percent < 0.0 and percent > 100.0) is tested implicitly + // by ensuring normal operation stays within bounds + normalResult := sampler.Sample(time.Second) + if normalResult < 0.0 || normalResult > 100.0 { + t.Errorf("CPU percentage %f is out of valid range [0, 100]", normalResult) + } +} diff --git a/internal/sysmonitor/cpu_fallback.go b/internal/sysmonitor/cpu_fallback.go new file mode 100644 index 0000000..fe54e7f --- /dev/null +++ b/internal/sysmonitor/cpu_fallback.go @@ -0,0 +1,11 @@ +//go:build !linux && !windows && !darwin + +package sysmonitor + +// newPlatformCPUSampler returns the heuristic sampler for unsupported OSs. +func newPlatformCPUSampler(_ FileSystem) (ProcessCPUSampler, error) { + // On unsupported platforms, we fall back to the GoroutineHeuristicSampler + // defined in cpu_heuristic.go. This ensures the application can still + // report an estimated "load" based on internal activity. + return NewGoroutineHeuristicSampler(), nil +} diff --git a/internal/sysmonitor/cpu_heuristic.go b/internal/sysmonitor/cpu_heuristic.go new file mode 100644 index 0000000..ba5c656 --- /dev/null +++ b/internal/sysmonitor/cpu_heuristic.go @@ -0,0 +1,74 @@ +package sysmonitor + +import ( + "math" + "runtime" + "time" +) + +const ( + // CPUHeuristicBaselineCPU provides minimum CPU usage estimate for any number of goroutines + CPUHeuristicBaselineCPU = 10.0 + // CPUHeuristicLinearScaleFactor determines CPU increase per goroutine for low counts (1-10) + CPUHeuristicLinearScaleFactor = 1.0 + // CPUHeuristicLogScaleFactor determines logarithmic CPU scaling for higher goroutine counts + CPUHeuristicLogScaleFactor = 8.0 + // CPUHeuristicMaxGoroutinesForLinear switches from linear to logarithmic scaling + CPUHeuristicMaxGoroutinesForLinear = 10 + // CPUHeuristicMaxCPU caps the CPU estimate to leave room for system processes + CPUHeuristicMaxCPU = 95.0 +) + +// GoroutineHeuristicSampler uses goroutine count as a CPU usage proxy. +// It implements the ProcessCPUSampler interface. +type GoroutineHeuristicSampler struct{} + +// NewGoroutineHeuristicSampler creates a new heuristic CPU sampler. +func NewGoroutineHeuristicSampler() *GoroutineHeuristicSampler { + return &GoroutineHeuristicSampler{} +} + +// Verify implementation of ProcessCPUSampler interface +var _ ProcessCPUSampler = &GoroutineHeuristicSampler{} + +// Sample returns the CPU usage percentage over the given time delta. +// Since this is a heuristic, the 'delta' argument is ignored as the +// calculation is based on instantaneous state (goroutine count). +func (s *GoroutineHeuristicSampler) Sample(_ time.Duration) float64 { + // Uses logarithmic scaling for more realistic CPU estimation + // Base level: 1-10 goroutines = baseline CPU usage (10-20%) + // Logarithmic growth to avoid overestimation at high goroutine counts + goroutineCount := float64(runtime.NumGoroutine()) + + // Baseline CPU usage for minimal goroutines + if goroutineCount <= CPUHeuristicMaxGoroutinesForLinear { + return CPUHeuristicBaselineCPU + goroutineCount*CPUHeuristicLinearScaleFactor + } + + // Logarithmic scaling: ln(goroutines) * scaling factor + // At ~100 goroutines: ~50% CPU + // At ~1000 goroutines: ~70% CPU + // At ~10000 goroutines: ~85% CPU + // Caps at 95% to leave room for system processes + logScaling := math.Log(goroutineCount) * CPUHeuristicLogScaleFactor + estimatedCPU := CPUHeuristicBaselineCPU + logScaling + + // Cap at maximum to be conservative + if estimatedCPU > CPUHeuristicMaxCPU { + return CPUHeuristicMaxCPU + } + return estimatedCPU +} + +// Reset prepares the sampler for a new sampling session. +// No-op for the heuristic sampler as it has no state. +func (s *GoroutineHeuristicSampler) Reset() { + // No state to reset for heuristic sampler + _ = s +} + +// IsInitialized returns true if the sampler has been initialized. +// Always true for heuristic sampler as it needs no baseline. +func (s *GoroutineHeuristicSampler) IsInitialized() bool { + return true +} diff --git a/internal/sysmonitor/cpu_heuristic_test.go b/internal/sysmonitor/cpu_heuristic_test.go new file mode 100644 index 0000000..46785a1 --- /dev/null +++ b/internal/sysmonitor/cpu_heuristic_test.go @@ -0,0 +1,96 @@ +package sysmonitor + +import ( + "runtime" + "sync" + "testing" + "time" +) + +func TestGoroutineHeuristicSampler(t *testing.T) { + sampler := NewGoroutineHeuristicSampler() + + // Test 1: Initialization + if !sampler.IsInitialized() { + t.Error("Heuristic sampler should always be initialized") + } + + // Test 2: Reset (should be no-op) + sampler.Reset() + if !sampler.IsInitialized() { + t.Error("Heuristic sampler should verify initialized after reset") + } + + // Test 3: Baseline Logic + // We expect at least the baseline CPU usage + val := sampler.Sample(time.Second) + if val < CPUHeuristicBaselineCPU { + t.Errorf("Expected baseline CPU >= %f, got %f", CPUHeuristicBaselineCPU, val) + } + + // Test 4: Scaling Logic + // We spawn a significant number of goroutines to force the count up + // and verify the "CPU usage" increases. + baseCount := runtime.NumGoroutine() + targetIncrease := 50 // Add 50 goroutines to trigger scaling + + var wg sync.WaitGroup + wg.Add(targetIncrease) + + // Channel to keep goroutines alive + hold := make(chan struct{}) + + for i := 0; i < targetIncrease; i++ { + go func() { + wg.Done() + <-hold + }() + } + + // Wait for all to start + wg.Wait() + + // Sample again with higher load + highLoadVal := sampler.Sample(time.Second) + + // Cleanup + close(hold) + + if highLoadVal <= val && baseCount < CPUHeuristicMaxGoroutinesForLinear { + t.Errorf("Expected higher CPU estimate with more goroutines. Low: %f, High: %f", val, highLoadVal) + } + + if highLoadVal > CPUHeuristicMaxCPU { + t.Errorf("CPU estimate %f exceeded max cap %f", highLoadVal, CPUHeuristicMaxCPU) + } + + // Test CPU cap is enforced - spawn many goroutines to trigger cap + manyGoroutines := 1000 // This should trigger logarithmic scaling and cap + var wg2 sync.WaitGroup + wg2.Add(manyGoroutines) + + // Channel to keep goroutines alive + hold2 := make(chan struct{}) + + for i := 0; i < manyGoroutines; i++ { + go func() { + wg2.Done() + <-hold2 + }() + } + + // Wait for all to start + wg2.Wait() + + // Sample with many goroutines - should be capped at CPUHeuristicMaxCPU + cappedVal := sampler.Sample(time.Second) + t.Logf("CPU estimate with %d goroutines: %f (max allowed: %f)", + manyGoroutines, cappedVal, CPUHeuristicMaxCPU) + if cappedVal > CPUHeuristicMaxCPU { + t.Errorf("CPU estimate %f exceeded max cap %f even with %d goroutines", + cappedVal, CPUHeuristicMaxCPU, manyGoroutines) + } + + // Cleanup + close(hold2) +} diff --git a/internal/sysmonitor/cpu_linux.go b/internal/sysmonitor/cpu_linux.go new file mode 100644 index 0000000..822f43e --- /dev/null +++ b/internal/sysmonitor/cpu_linux.go @@ -0,0 +1,207 @@ +//go:build linux + +package sysmonitor + +import ( + "bytes" + "encoding/binary" + "fmt" + "math" + "os" + "runtime" + "strconv" + "strings" + "time" +) + +// linuxProcessSampler samples CPU usage for the current process on Linux +type linuxProcessSampler struct { + fs FileSystem + pid int + lastUTime float64 + lastSTime float64 + lastSample time.Time + lastPercent float64 + clockTicks int64 +} + +// newPlatformCPUSampler is the factory entry point used by cpu.go +func newPlatformCPUSampler(fs FileSystem) (ProcessCPUSampler, error) { + pid := os.Getpid() + if pid < 0 || pid > math.MaxInt32 { + return nil, fmt.Errorf("invalid PID: %d", pid) + } + + // We attempt to get clock ticks using the provided FS. + // If it fails, we fallback to 100. + ticks, err := getClockTicks(fs) + if err != nil { + ticks = 100 + } + + return &linuxProcessSampler{ + fs: fs, + pid: pid, + clockTicks: ticks, + }, nil +} + +// Sample returns the CPU usage percentage since the last sample +func (s *linuxProcessSampler) Sample(deltaTime time.Duration) float64 { + now := time.Now() + if s.lastSample.IsZero() { + // First sample - try to initialize + utime, stime, err := s.readProcessTimes() + if err != nil { + // If we can't read process times, still mark as initialized to avoid repeated attempts + s.lastSample = now + s.lastPercent = 0.0 + return 0.0 + } + s.lastUTime = float64(utime) + s.lastSTime = float64(stime) + s.lastSample = now + s.lastPercent = 0.0 + return 0.0 + } + + utime, stime, err := s.readProcessTimes() + if err != nil { + return s.lastPercent // Return last known value on error + } + + elapsed := now.Sub(s.lastSample) + // If called too frequently, return cached value to avoid jitter + if elapsed < deltaTime/2 { + return s.lastPercent + } + + // Convert ticks to seconds and calculate CPU usage + prevTotalTime := s.lastUTime + s.lastSTime + currTotalTime := float64(utime) + float64(stime) + cpuTimeDelta := currTotalTime - prevTotalTime + cpuTimeSeconds := cpuTimeDelta / float64(s.clockTicks) + wallTimeSeconds := elapsed.Seconds() + + // Normalized to 0-100% (divides by numCPU for system-wide metric) + numcpu := runtime.NumCPU() + if numcpu <= 0 { + numcpu = 1 // Safety check + } + + if wallTimeSeconds <= 0 { + return s.lastPercent + } + + percent := (cpuTimeSeconds / wallTimeSeconds) * 100.0 / float64(numcpu) + + if percent > 100.0 { + percent = 100.0 + } else if percent < 0.0 { + percent = 0.0 + } + s.lastUTime = float64(utime) + s.lastSTime = float64(stime) + s.lastSample = now + s.lastPercent = percent + + return percent +} + +// Reset clears sampler state for a new session +func (s *linuxProcessSampler) Reset() { + s.lastUTime = 0.0 + s.lastSTime = 0.0 + s.lastSample = time.Time{} + s.lastPercent = 0.0 +} + +// IsInitialized returns true if at least one sample has been taken +func (s *linuxProcessSampler) IsInitialized() bool { + return !s.lastSample.IsZero() +} + +// readProcessTimes reads CPU times from /proc//stat (returns ticks) +func (s *linuxProcessSampler) readProcessTimes() (utime, stime int64, err error) { + path := fmt.Sprintf("/proc/%d/stat", s.pid) + + content, err := s.fs.ReadFile(path) + if err != nil { + return 0, 0, fmt.Errorf("failed to read file %s: %w", path, err) + } + + fields := strings.Fields(string(content)) + if len(fields) < 17 { + return 0, 0, fmt.Errorf( + "invalid stat file format for /proc/%d/stat: expected at least 17 fields, got %d", + s.pid, len(fields)) + } + + // utime=field[13], stime=field[14] + utime, err = strconv.ParseInt(fields[13], 10, 64) + if err != nil { + return 0, 0, fmt.Errorf("failed to parse utime from field[13]: %w", err) + } + + stime, err = strconv.ParseInt(fields[14], 10, 64) + if err != nil { + return 0, 0, fmt.Errorf("failed to parse stime from field[14]: %w", err) + } + + return utime, stime, nil +} + +// getClockTicks reads clock ticks per second from /proc/self/auxv (AT_CLKTCK=17) +func getClockTicks(fs FileSystem) (int64, error) { + data, err := fs.ReadFile("/proc/self/auxv") + if err != nil { + return 100, nil // fallback + } + + // Try 64-bit format first (16 bytes per entry) + if len(data)%16 == 0 { + buf := bytes.NewReader(data) + var id, val uint64 + + for { + if err := binary.Read(buf, binary.LittleEndian, &id); err != nil { + break + } + if err := binary.Read(buf, binary.LittleEndian, &val); err != nil { + break + } + + if id == 17 && val > 0 && val <= 10000 { // AT_CLKTCK + return int64(val), nil + } + } + } + + // Try 32-bit format (8 bytes per entry) + if len(data)%8 == 0 { + return parseAuxv32(data) + } + + return 100, nil // fallback +} + +// parseAuxv32 parses 32-bit auxv format +func parseAuxv32(data []byte) (int64, error) { + buf := bytes.NewReader(data) + var id, val uint32 + + for { + if err := binary.Read(buf, binary.LittleEndian, &id); err != nil { + break + } + if err := binary.Read(buf, binary.LittleEndian, &val); err != nil { + break + } + + if id == 17 && val > 0 && val <= 10000 { // AT_CLKTCK + return int64(val), nil + } + } + + return 100, nil +} diff --git a/internal/sysmonitor/cpu_linux_test.go b/internal/sysmonitor/cpu_linux_test.go new file mode 100644 index 0000000..929d825 --- /dev/null +++ b/internal/sysmonitor/cpu_linux_test.go @@ -0,0 +1,220 @@ +//go:build linux + +package sysmonitor + +import ( + "bytes" + "encoding/binary" + "fmt" + "testing" + "time" +) + +// generateAuxv creates a binary payload mimicking /proc/self/auxv +// ID 17 = AT_CLKTCK +func generateAuxv(ticks uint64) []byte { + buf := new(bytes.Buffer) + // Write AT_CLKTCK entry + binary.Write(buf, binary.LittleEndian, uint64(17)) // ID + binary.Write(buf, binary.LittleEndian, ticks) // Value + // Write null entry to end list + binary.Write(buf, binary.LittleEndian, uint64(0)) + binary.Write(buf, binary.LittleEndian, uint64(0)) + return buf.Bytes() +} + +func TestLinuxProcessSampler(t *testing.T) { + // Setup Mock FS + mockFS := &MockFileSystem{ + Files: map[string][]byte{ + "/proc/self/auxv": generateAuxv(100), // 100 ticks per second + }, + } + + // Test Factory + samplerInterface, err := newPlatformCPUSampler(mockFS) + if err != nil { + t.Fatalf("Failed to create sampler: %v", err) + } + sampler := samplerInterface.(*linuxProcessSampler) + + // Verify Ticks parsing + if sampler.clockTicks != 100 { + t.Errorf("Expected 100 clock ticks, got %d", sampler.clockTicks) + } + + // Setup Test Data + // format: pid (comm) state ppid pgrp session tty_nr tpgid flags minflt cminflt majflt cmajflt utime stime ... + // Fields 13 (utime) and 14 (stime) are 0-indexed in fields array (so indices 13 and 14) + // Actually strings.Fields 1-based index in documentation usually maps to: + // 13: utime, 14: stime. + // Let's construct a valid line. "100 (go) R 1 1 1 0 -1 4194304 100 0 0 0 10 20 0 0 ..." + // ticks = 10 + 20 = 30 total ticks + statContent1 := []byte("100 (test) R 1 1 1 0 -1 0 0 0 0 0 10 20 0 0 20 0 1 0 0 0 0 0 0 0 0 0 0 0 0") + mockFS.Files[fmt.Sprintf("/proc/%d/stat", sampler.pid)] = statContent1 + + // 1. First Sample (Initialization) + val := sampler.Sample(time.Second) + if val != 0.0 { + t.Errorf("First sample should be 0.0, got %f", val) + } + if !sampler.IsInitialized() { + t.Error("Sampler should be initialized after first call") + } + + // 2. Second Sample (Activity) + // Increase ticks by 50 (User) + 50 (System) = 100 ticks delta + // 100 ticks / 100 ticks/sec = 1 second CPU time + statContent2 := []byte("100 (test) R 1 1 1 0 -1 0 0 0 0 0 60 70 0 0 20 0 1 0 0 0 0 0 0 0 0 0 0 0 0") + mockFS.Files[fmt.Sprintf("/proc/%d/stat", sampler.pid)] = statContent2 + + // Sleep briefly to allow Sample's timing logic to progress. + // This simulates passage of real time for the test. + time.Sleep(50 * time.Millisecond) + val = sampler.Sample(0) // 0 delta to force calculation + + if val <= 0 { + t.Errorf("Expected CPU usage > 0, got %f", val) + } + + // 3. Test Reset + sampler.Reset() + if sampler.IsInitialized() { + t.Error("Sampler should not be initialized after Reset") + } + + // 4. Test Bad File + mockFS.Files[fmt.Sprintf("/proc/%d/stat", sampler.pid)] = []byte("garbage") + val = sampler.Sample(time.Second) + // Should return last known good value (or 0 if reset) + if val != 0.0 { + t.Errorf("Expected 0.0 on error after reset, got %f", val) + } +} + +func TestGetClockTicksFallback(t *testing.T) { + mockFS := &MockFileSystem{Files: map[string][]byte{}} // Empty FS + + ticks, err := getClockTicks(mockFS) + if err != nil { + t.Errorf("Expected no error on fallback, got %v", err) + } + if ticks != 100 { + t.Errorf("Expected fallback ticks 100, got %d", ticks) + } +} + +// generateAuxv32 creates a binary payload mimicking 32-bit /proc/self/auxv +func generateAuxv32(ticks uint32) []byte { + buf := new(bytes.Buffer) + // Write AT_CLKTCK entry (32-bit format) + binary.Write(buf, binary.LittleEndian, uint32(17)) // ID + binary.Write(buf, binary.LittleEndian, ticks) // Value + // Write null entry to end list + binary.Write(buf, binary.LittleEndian, uint32(0)) + binary.Write(buf, binary.LittleEndian, uint32(0)) + return buf.Bytes() +} + +func TestParseAuxv32(t *testing.T) { + tests := []struct { + name string + data []byte + expected int64 + }{ + { + name: "Valid 32-bit auxv with ticks", + data: generateAuxv32(250), + expected: 250, + }, + { + name: "Valid 32-bit auxv with different ticks", + data: generateAuxv32(500), + expected: 500, + }, + { + name: "32-bit auxv with invalid ticks (too high)", + data: generateAuxv32(20000), // > 10000, should be ignored + expected: 100, // fallback + }, + { + name: "32-bit auxv with zero ticks", + data: generateAuxv32(0), // should be ignored + expected: 100, // fallback + }, + { + name: "Empty auxv data", + data: []byte{}, + expected: 100, // fallback + }, + { + name: "Malformed auxv data (odd length)", + data: []byte{1, 2, 3}, // not multiple of 8 + expected: 100, // fallback + }, + { + name: "32-bit auxv with wrong ID", + data: func() []byte { + buf := new(bytes.Buffer) + binary.Write(buf, binary.LittleEndian, uint32(99)) // Wrong ID + binary.Write(buf, binary.LittleEndian, uint32(250)) + binary.Write(buf, binary.LittleEndian, uint32(0)) + binary.Write(buf, binary.LittleEndian, uint32(0)) + return buf.Bytes() + }(), + expected: 100, // fallback + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + result, err := parseAuxv32(tt.data) + if err != nil { + t.Errorf("parseAuxv32() error = %v", err) + return + } + if result != tt.expected { + t.Errorf("parseAuxv32() = %v, expected %v", result, tt.expected) + } + }) + } +} + +func TestGetClockTicks32Bit(t *testing.T) { + mockFS := &MockFileSystem{ + Files: map[string][]byte{ + "/proc/self/auxv": generateAuxv32(250), // 32-bit format (8 bytes per entry) + }, + } + + ticks, err := getClockTicks(mockFS) + if err != nil { + t.Errorf("Expected no error, got %v", err) + } + if ticks != 250 { + t.Errorf("Expected 250 ticks, got %d", ticks) + } +} + +func TestNewPlatformCPUSamplerErrors(t *testing.T) { + // Test clock ticks read failure (should still succeed with fallback) + mockFS := &MockFileSystem{ + OpenErrs: map[string]error{ + "/proc/self/auxv": fmt.Errorf("permission denied"), + }, + } + + sampler, err := newPlatformCPUSampler(mockFS) + if err != nil { + t.Errorf("Expected no error with fallback, got %v", err) + } + if sampler == nil { + t.Error("Expected sampler to be created") + } + + // Verify fallback ticks were used + linuxSampler := sampler.(*linuxProcessSampler) + if linuxSampler.clockTicks != 100 { + t.Errorf("Expected fallback ticks 100, got %d", linuxSampler.clockTicks) + } +} diff --git a/internal/sysmonitor/cpu_test.go b/internal/sysmonitor/cpu_test.go new file mode 100644 index 0000000..807a970 --- /dev/null +++ b/internal/sysmonitor/cpu_test.go @@ -0,0 +1,182 @@ +package sysmonitor + +import ( + "io/fs" + "testing" + "time" +) + +// MockFS for the factory test +type factoryMockFS struct{} + +func (f *factoryMockFS) ReadFile(_ string) ([]byte, error) { return nil, nil } +func (f *factoryMockFS) Open(_ string) (fs.File, error) { return nil, nil } + +type testError struct { + msg string +} + +func (e *testError) Error() string { + return e.msg +} + +func TestNewCPUSampler(t *testing.T) { + fs := &factoryMockFS{} + + sampler, err := NewCPUSampler(fs) + if err != nil { + t.Fatalf("NewCPUSampler returned error: %v", err) + } + + if sampler == nil { + t.Fatal("NewCPUSampler returned nil") + } + + // Basic interface check + sampler.Reset() + _ = sampler.IsInitialized() // Ensure method exists and is callable +} + +// TestProcessCPUSamplerInterface tests the ProcessCPUSampler interface +// using any available platform implementation +func TestProcessCPUSamplerInterface(t *testing.T) { + // Create a mock filesystem for testing + fs := &factoryMockFS{} + + sampler, err := NewCPUSampler(fs) + if err != nil { + t.Skipf("CPU sampler not available on this platform: %v", err) + } + + // Test 1: Interface compliance - verify all methods exist and are callable + if sampler == nil { + t.Fatal("NewCPUSampler returned nil sampler") + } + + // Test 2: Initial state - should not be initialized + if sampler.IsInitialized() { + t.Error("New sampler should not be initialized") + } + + // Test 3: First sample - should initialize and return 0.0 + firstSample := sampler.Sample(time.Second) + if !sampler.IsInitialized() { + t.Error("Sampler should be initialized after first sample") + } + if firstSample != 0.0 { + t.Errorf("First sample should return 0.0, got %f", firstSample) + } + + // Test 4: Subsequent samples - should return valid CPU values + secondSample := sampler.Sample(time.Second) + if secondSample < 0.0 || secondSample > 100.0 { + t.Errorf("CPU sample should be between 0-100, got %f", secondSample) + } + + // Test 5: Reset functionality + sampler.Reset() + if sampler.IsInitialized() { + t.Error("Sampler should not be initialized after reset") + } + + // Test 6: Sample after reset - should behave like first sample + resetSample := sampler.Sample(time.Second) + if !sampler.IsInitialized() { + t.Error("Sampler should be initialized after sample following reset") + } + if resetSample != 0.0 { + t.Errorf("Sample after reset should return 0.0, got %f", resetSample) + } + + // Test 7: Rapid sampling - should return last known value if delta too small + rapidSample := sampler.Sample(time.Millisecond) + // This should return the last known value (resetSample) since delta is too small + if rapidSample != resetSample { + t.Logf("Rapid sample returned %f, expected last value %f", rapidSample, resetSample) + // This is informational - behavior may vary by implementation + } +} + +// TestProcessMemoryReaderInterface tests the ProcessMemoryReader interface +// using any available platform implementation +func TestProcessMemoryReaderInterface(t *testing.T) { + // Create a mock filesystem for testing + fs := &factoryMockFS{} + + reader := NewProcessMemoryReader(fs) + if reader == nil { + t.Fatal("NewProcessMemoryReader returned nil reader") + } + + // Test 1: Basic sampling functionality + mem, err := reader.Sample() + if err != nil { + t.Skipf("Memory reader not functional on this platform: %v", err) + } + + // Test 2: Memory values should be reasonable + if mem.Total == 0 { + t.Error("Total memory should not be zero") + } + if mem.Available > mem.Total { + t.Errorf("Available memory (%d) should not exceed total memory (%d)", mem.Available, mem.Total) + } + + // Test 3: Multiple samples should be consistent + mem2, err := reader.Sample() + if err != nil { + t.Errorf("Second sample failed: %v", err) + } + + // Total memory should be consistent + if mem2.Total != mem.Total { + t.Errorf("Total memory changed between samples: %d -> %d", mem.Total, mem2.Total) + } + + // Available memory should be reasonable (within 10% of previous value) + upperBound := uint64(float64(mem.Available) * 1.1) + lowerBound := uint64(float64(mem.Available) * 0.9) + if mem2.Available > upperBound || mem2.Available < lowerBound { + t.Logf("Available memory changed significantly: %d -> %d", mem.Available, mem2.Available) + // This is informational as memory usage can fluctuate + } +} + +// TestSamplerErrorHandling tests error conditions that work across platforms +func TestSamplerErrorHandling(t *testing.T) { + // Test with mock filesystem that can simulate errors + mockFS := &MockFileSystem{ + OpenErrs: map[string]error{ + "/proc/self/stat": &testError{msg: "mock stat error"}, + "/proc/meminfo": &testError{msg: "mock meminfo error"}, + "/proc/self/auxv": &testError{msg: "mock auxv error"}, + }, + } + + // Test CPU sampler with error conditions + cpuSampler, err := NewCPUSampler(mockFS) + if err == nil { + t.Log("CPU sampler creation succeeded despite mock errors - this may be expected on some platforms") + // If it succeeds, test that it still functions + if cpuSampler != nil { + sample := cpuSampler.Sample(time.Second) + if sample < 0.0 { + t.Errorf("CPU sample should not be negative: %f", sample) + } + } + } else { + t.Logf("CPU sampler creation failed as expected: %v", err) + } + + // Test memory reader with error conditions + memReader := NewProcessMemoryReader(mockFS) + if memReader != nil { + _, err := memReader.Sample() + // Error is expected but not guaranteed on all platforms + if err != nil { + t.Logf("Memory reader returned expected error: %v", err) + } else { + t.Log("Memory reader succeeded despite mock errors - this may be expected on some platforms") + } + } +} diff --git a/internal/sysmonitor/cpu_windows.go b/internal/sysmonitor/cpu_windows.go new file mode 100644 index 0000000..781d22c --- /dev/null +++ b/internal/sysmonitor/cpu_windows.go @@ -0,0 +1,159 @@ +//go:build windows + +package sysmonitor + +import ( + "fmt" + "math" + "os" + "runtime" + "syscall" + "time" +) + +// windowsProcessSampler samples CPU usage for the current process on Windows +type windowsProcessSampler struct { + pid int + lastUTime float64 + lastSTime float64 + lastSample time.Time + lastPercent float64 +} + +// newPlatformCPUSampler matches the factory signature required by cpu.go. +func newPlatformCPUSampler(_ FileSystem) (ProcessCPUSampler, error) { + pid := os.Getpid() + if pid < 0 || pid > math.MaxInt32 { + return nil, fmt.Errorf("invalid PID: %d", pid) + } + + return &windowsProcessSampler{ + pid: pid, + }, nil +} + +// Sample returns the CPU usage percentage since the last sample +func (s *windowsProcessSampler) Sample(deltaTime time.Duration) float64 { + utime, stime, err := s.getCurrentCPUTimes() + if err != nil { + // If we have a previous valid sample, return it; otherwise return 0 + if s.lastSample.IsZero() { + return 0.0 + } + return s.lastPercent + } + + now := time.Now() + if s.lastSample.IsZero() { + s.lastUTime = utime + s.lastSTime = stime + s.lastSample = now + s.lastPercent = 0.0 + return 0.0 + } + + elapsed := now.Sub(s.lastSample) + if elapsed < deltaTime/2 { + return s.lastPercent + } + + // GetProcessTimes returns sum of all threads across all cores + cpuTimeDelta := (utime + stime) - (s.lastUTime + s.lastSTime) + wallTimeSeconds := elapsed.Seconds() + + if wallTimeSeconds <= 0 { + return s.lastPercent + } + + // Normalized to 0-100% (divides by numCPU for system-wide metric) + numcpu := runtime.NumCPU() + if numcpu <= 0 { + numcpu = 1 // Safety check + } + + percent := (cpuTimeDelta / wallTimeSeconds) * 100.0 / float64(numcpu) + + // Handle negative deltas (can happen due to clock adjustments or process restarts) + if percent < 0.0 { + percent = 0.0 + } else if percent > 100.0 { + percent = 100.0 + } + + s.lastUTime = utime + s.lastSTime = stime + s.lastSample = now + s.lastPercent = percent + + return percent +} + +// Reset clears sampler state for a new session +func (s *windowsProcessSampler) Reset() { + s.lastUTime = 0.0 + s.lastSTime = 0.0 + s.lastSample = time.Time{} + s.lastPercent = 0.0 +} + +// IsInitialized returns true if at least one sample has been taken +func (s *windowsProcessSampler) IsInitialized() bool { + return !s.lastSample.IsZero() +} + +// getProcessCPUTimes retrieves CPU times via GetProcessTimes (returns FILETIME) +func getProcessCPUTimes(pid int) (syscall.Filetime, syscall.Filetime, error) { + var c, e, k, u syscall.Filetime + + currentPid := os.Getpid() + var h syscall.Handle + if pid == currentPid { + var err error + h, err = syscall.GetCurrentProcess() + if err != nil { + return k, u, fmt.Errorf("failed to get current process handle: %w", err) + } + } else { + // Try PROCESS_QUERY_LIMITED_INFORMATION first (works on more Windows versions) + // Fall back to PROCESS_QUERY_INFORMATION if that fails + var err error + // Define constant here since we can't depend on x/sys/windows + const PROCESS_QUERY_LIMITED_INFORMATION = 0x1000 + h, err = syscall.OpenProcess(PROCESS_QUERY_LIMITED_INFORMATION, false, uint32(pid)) + if err != nil { + // Fallback to PROCESS_QUERY_INFORMATION + h, err = syscall.OpenProcess(syscall.PROCESS_QUERY_INFORMATION, false, uint32(pid)) + if err != nil { + return k, u, fmt.Errorf("failed to open process %d: %w", pid, err) + } + } + defer syscall.CloseHandle(h) + } + + err := syscall.GetProcessTimes(h, &c, &e, &k, &u) + if err != nil { + return k, u, fmt.Errorf("failed to get process times for PID %d: %w", pid, err) + } + + return k, u, nil +} + +// convertFiletimeToSeconds converts FILETIME (100ns intervals) to seconds +func convertFiletimeToSeconds(ft syscall.Filetime) float64 { + // Join high/low into single 64-bit integer + ticks := int64(ft.HighDateTime)<<32 | int64(ft.LowDateTime) + // 1 tick = 100ns = 0.0000001 seconds + return float64(ticks) * 1e-7 +} + +// getCurrentCPUTimes reads CPU times for the process (returns seconds) +func (s *windowsProcessSampler) getCurrentCPUTimes() (utime, stime float64, err error) { + k, u, err := getProcessCPUTimes(s.pid) + if err != nil { + return 0, 0, fmt.Errorf("failed to get CPU times for process %d: %w", s.pid, err) + } + + utime = convertFiletimeToSeconds(u) + stime = convertFiletimeToSeconds(k) + return utime, stime, nil +} diff --git a/internal/sysmonitor/cpu_windows_test.go b/internal/sysmonitor/cpu_windows_test.go new file mode 100644 index 0000000..d392994 --- /dev/null +++ b/internal/sysmonitor/cpu_windows_test.go @@ -0,0 +1,291 @@ +//go:build windows + +package sysmonitor + +import ( + "testing" + "time" +) + +// TestWindowsSamplerIntegration runs real calls against the OS. +func TestWindowsSamplerIntegration(t *testing.T) { + sampler, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Windows sampler: %v", err) + } + + // 1. Initialize + val := sampler.Sample(time.Second) + if val != 0.0 { + t.Errorf("Expected initial sample 0.0, got %f", val) + } + + // 2. Generate Load (Busy Loop) to ensure non-zero CPU + done := make(chan struct{}) + go func() { + end := time.Now().Add(100 * time.Millisecond) + for time.Now().Before(end) { + } + close(done) + }() + <-done + + // Sleep a tiny bit to ensure total elapsed > busy loop time + time.Sleep(50 * time.Millisecond) + + // 3. Measure + val = sampler.Sample(0) + + // Log result (useful for verification) + t.Logf("Measured CPU Load: %f%%", val) + + if val <= 0.0 { + t.Error("Expected non-zero CPU usage after busy loop") + } + if val > 100.0 { + t.Errorf("CPU usage %f exceeds 100%%", val) + } +} + +// TestWindowsSamplerReset tests the Reset functionality +func TestWindowsSamplerReset(t *testing.T) { + sampler, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Windows sampler: %v", err) + } + + // Take an initial sample to initialize state + val1 := sampler.Sample(time.Second) + if val1 != 0.0 { + t.Errorf("Expected initial sample 0.0, got %f", val1) + } + + // Verify sampler is initialized + if !sampler.IsInitialized() { + t.Error("Expected sampler to be initialized after first sample") + } + + // Reset the sampler + sampler.Reset() + + // Verify sampler is no longer initialized + if sampler.IsInitialized() { + t.Error("Expected sampler to be uninitialized after reset") + } + + // Take another sample - should return 0.0 again (like first sample) + val2 := sampler.Sample(time.Second) + if val2 != 0.0 { + t.Errorf("Expected sample after reset to be 0.0, got %f", val2) + } + + // Verify sampler is now initialized again + if !sampler.IsInitialized() { + t.Error("Expected sampler to be initialized after second sample") + } +} + +func TestWindowsSamplerIsInitialized(t *testing.T) { + sampler, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Windows sampler: %v", err) + } + + // Initially should not be initialized + if sampler.IsInitialized() { + t.Error("Expected new sampler to be uninitialized") + } + + // After first sample, should be initialized + sampler.Sample(time.Second) + if !sampler.IsInitialized() { + t.Error("Expected sampler to be initialized after first sample") + } + + // After reset, should be uninitialized again + sampler.Reset() + if sampler.IsInitialized() { + t.Error("Expected sampler to be uninitialized after reset") + } +} + +func TestWindowsSamplerShortInterval(t *testing.T) { + sampler, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Windows sampler: %v", err) + } + + // Initialize with first sample + val1 := sampler.Sample(time.Second) + if val1 != 0.0 { + t.Errorf("Expected initial sample 0.0, got %f", val1) + } + + // Generate some CPU load and take a normal sample + done := make(chan struct{}) + go func() { + end := time.Now().Add(50 * time.Millisecond) + for time.Now().Before(end) { + } + close(done) + }() + <-done + + time.Sleep(10 * time.Millisecond) // Ensure some elapsed time + val2 := sampler.Sample(100 * time.Millisecond) // Normal sample + t.Logf("Normal sample after load: %f", val2) + + // Now test short interval behavior - sample immediately again with very short delta + val3 := sampler.Sample(10 * time.Millisecond) // Very short interval (< deltaTime/2 = 5ms) + + // Should return the previous value due to short interval + if val3 != val2 { + t.Errorf("Expected same value %f due to short interval (10ms < 50ms), got %f", val2, val3) + } else { + t.Logf("Short interval correctly returned cached value: %f", val3) + } +} + +func TestWindowsSamplerConsistency(t *testing.T) { + sampler, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Windows sampler: %v", err) + } + + // Initialize + sampler.Sample(time.Second) + + // Take multiple samples over time + var samples []float64 + for i := 0; i < 3; i++ { + time.Sleep(100 * time.Millisecond) + val := sampler.Sample(50 * time.Millisecond) + samples = append(samples, val) + + if val < 0.0 || val > 100.0 { + t.Errorf("Sample %d out of valid range: %f", i, val) + } + } + + t.Logf("Consistency test samples: %v", samples) +} + +// TestWindowsSamplerBoundaryCalculations tests edge cases in CPU calculations +func TestWindowsSamplerBoundaryCalculations(t *testing.T) { + sampler, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Windows sampler: %v", err) + } + + // Test many rapid samples to ensure bounds checking works + for i := 0; i < 20; i++ { + val := sampler.Sample(time.Millisecond) + if val < 0.0 { + t.Errorf("Sample %d returned negative value: %f", i, val) + } + if val > 100.0 { + t.Errorf("Sample %d exceeded 100%%: %f", i, val) + } + } + + // Test with very small delta times + val := sampler.Sample(time.Nanosecond) + if val < 0.0 || val > 100.0 { + t.Errorf("Nanosecond delta sample out of bounds: %f", val) + } + + // Test reset and immediate sampling + sampler.Reset() + val = sampler.Sample(0) + if val != 0.0 { + t.Errorf("Expected 0.0 after reset, got %f", val) + } +} + +// TestWindowsSamplerLoadScenarios tests various CPU load scenarios +func TestWindowsSamplerLoadScenarios(t *testing.T) { + sampler, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Windows sampler: %v", err) + } + + // Initialize + sampler.Sample(time.Second) + + // Test 1: No load scenario + time.Sleep(50 * time.Millisecond) + val1 := sampler.Sample(100 * time.Millisecond) + t.Logf("No load CPU: %f%%", val1) + + // Test 2: Light load scenario + done := make(chan struct{}) + go func() { + end := time.Now().Add(30 * time.Millisecond) + for time.Now().Before(end) { + // Light CPU work + _ = time.Now().UnixNano() + } + close(done) + }() + <-done + + time.Sleep(10 * time.Millisecond) + val2 := sampler.Sample(100 * time.Millisecond) + t.Logf("Light load CPU: %f%%", val2) + + // Test 3: Moderate load scenario + done = make(chan struct{}) + go func() { + end := time.Now().Add(50 * time.Millisecond) + for time.Now().Before(end) { + // Moderate CPU work + for j := 0; j < 1000; j++ { + _ = j * j + } + } + close(done) + }() + <-done + + time.Sleep(10 * time.Millisecond) + val3 := sampler.Sample(100 * time.Millisecond) + t.Logf("Moderate load CPU: %f%%", val3) + + // All values should be valid + for i, val := range []float64{val1, val2, val3} { + if val < 0.0 || val > 100.0 { + t.Errorf("Load scenario %d out of bounds: %f", i+1, val) + } + } +} + +// TestWindowsSamplerTimingBehavior tests timing-related behavior +func TestWindowsSamplerTimingBehavior(t *testing.T) { + sampler, err := newPlatformCPUSampler(nil) + if err != nil { + t.Fatalf("Failed to create Windows sampler: %v", err) + } + + // Initialize + start := time.Now() + sampler.Sample(time.Second) + initTime := time.Since(start) + + // Test rapid sampling behavior + for i := 0; i < 5; i++ { + start := time.Now() + val := sampler.Sample(50 * time.Millisecond) + elapsed := time.Since(start) + + // Should complete quickly (less than 10ms typically) + if elapsed > 100*time.Millisecond { + t.Errorf("Sample %d took too long: %v", i, elapsed) + } + + if val < 0.0 || val > 100.0 { + t.Errorf("Sample %d value out of bounds: %f", i, val) + } + } + + t.Logf("Initialization took: %v", initTime) +} diff --git a/internal/sysmonitor/doc.go b/internal/sysmonitor/doc.go new file mode 100644 index 0000000..1f17dbd --- /dev/null +++ b/internal/sysmonitor/doc.go @@ -0,0 +1,3 @@ +// Package sysmonitor provides system monitoring functionality +// for CPU and memory usage monitoring. +package sysmonitor diff --git a/internal/sysmonitor/fs.go b/internal/sysmonitor/fs.go new file mode 100644 index 0000000..8023654 --- /dev/null +++ b/internal/sysmonitor/fs.go @@ -0,0 +1,23 @@ +package sysmonitor + +import ( + "io/fs" + "os" +) + +// FileSystem abstracts file system operations for testing +type FileSystem interface { + ReadFile(name string) ([]byte, error) + Open(name string) (fs.File, error) +} + +// OSFileSystem implements FileSystem using the os package +type OSFileSystem struct{} + +func (OSFileSystem) ReadFile(name string) ([]byte, error) { + return os.ReadFile(name) +} + +func (OSFileSystem) Open(name string) (fs.File, error) { + return os.Open(name) +} diff --git a/internal/sysmonitor/fs_test.go b/internal/sysmonitor/fs_test.go new file mode 100644 index 0000000..875c03e --- /dev/null +++ b/internal/sysmonitor/fs_test.go @@ -0,0 +1,187 @@ +package sysmonitor + +import ( + "errors" + "io/fs" + "os" + "path/filepath" + "testing" +) + +// MockFS is a mock implementation of FileSystem for testing +type MockFS struct { + ReadFileFunc func(name string) ([]byte, error) + OpenFunc func(name string) (fs.File, error) +} + +func (m MockFS) ReadFile(name string) ([]byte, error) { + if m.ReadFileFunc != nil { + return m.ReadFileFunc(name) + } + return nil, nil +} + +func (m MockFS) Open(name string) (fs.File, error) { + if m.OpenFunc != nil { + return m.OpenFunc(name) + } + return nil, nil +} + +func TestOSFileSystem_ReadFile(t *testing.T) { + // Create a temporary directory for testing + tempDir := t.TempDir() + + // Test successful read + testContent := "Hello, World!" + testFile := filepath.Join(tempDir, "test.txt") + err := os.WriteFile(testFile, []byte(testContent), 0o600) + if err != nil { + t.Fatalf("Failed to create test file: %v", err) + } + + fs := OSFileSystem{} + data, err := fs.ReadFile(testFile) + if err != nil { + t.Errorf("ReadFile failed: %v", err) + } + if string(data) != testContent { + t.Errorf("Expected %q, got %q", testContent, string(data)) + } + + // Test reading non-existent file + _, err = fs.ReadFile(filepath.Join(tempDir, "nonexistent.txt")) + if err == nil { + t.Error("Expected error when reading non-existent file, got nil") + } + if !os.IsNotExist(err) { + t.Errorf("Expected IsNotExist error, got %v", err) + } +} + +func TestOSFileSystem_Open(t *testing.T) { + // Create a temporary directory for testing + tempDir := t.TempDir() + + // Test successful open + testContent := "Hello, World!" + testFile := filepath.Join(tempDir, "test.txt") + err := os.WriteFile(testFile, []byte(testContent), 0o600) + if err != nil { + t.Fatalf("Failed to create test file: %v", err) + } + + fs := OSFileSystem{} + file, err := fs.Open(testFile) + if err != nil { + t.Errorf("Open failed: %v", err) + } + defer file.Close() + + // Verify we can read from the opened file + data := make([]byte, len(testContent)) + n, err := file.Read(data) + if err != nil { + t.Errorf("Failed to read from opened file: %v", err) + } + if n != len(testContent) { + t.Errorf("Expected to read %d bytes, got %d", len(testContent), n) + } + if string(data) != testContent { + t.Errorf("Expected %q, got %q", testContent, string(data)) + } + + // Test opening non-existent file + _, err = fs.Open(filepath.Join(tempDir, "nonexistent.txt")) + if err == nil { + t.Error("Expected error when opening non-existent file, got nil") + } + if !os.IsNotExist(err) { + t.Errorf("Expected IsNotExist error, got %v", err) + } +} + +func TestMockFS_ReadFile(t *testing.T) { + expectedData := []byte("mock data") + expectedErr := os.ErrNotExist + + mock := MockFS{ + ReadFileFunc: func(name string) ([]byte, error) { + if name == "success.txt" { + return expectedData, nil + } + return nil, expectedErr + }, + } + + // Test success case + data, err := mock.ReadFile("success.txt") + if err != nil { + t.Errorf("Expected no error, got %v", err) + } + if string(data) != string(expectedData) { + t.Errorf("Expected %q, got %q", expectedData, data) + } + + // Test error case + _, err = mock.ReadFile("error.txt") + if !errors.Is(err, expectedErr) { + t.Errorf("Expected error %v, got %v", expectedErr, err) + } + + // Test default behavior (no function set) + mockDefault := MockFS{} + data, err = mockDefault.ReadFile("any.txt") + if err != nil { + t.Errorf("Expected no error for default mock, got %v", err) + } + if data != nil { + t.Errorf("Expected nil data for default mock, got %v", data) + } +} + +func TestMockFS_Open(t *testing.T) { + expectedErr := os.ErrNotExist + + mock := MockFS{ + OpenFunc: func(name string) (fs.File, error) { + if name == "error.txt" { + return nil, expectedErr + } + return nil, nil // Mock file (nil for simplicity in tests) + }, + } + + // Test error case + _, err := mock.Open("error.txt") + if !errors.Is(err, expectedErr) { + t.Errorf("Expected error %v, got %v", expectedErr, err) + } + + // Test success case (returns nil file for simplicity) + file, err := mock.Open("success.txt") + if err != nil { + t.Errorf("Expected no error, got %v", err) + } + if file != nil { + t.Errorf("Expected nil file for mock, got %v", file) + } + + // Test default behavior (no function set) + mockDefault := MockFS{} + file, err = mockDefault.Open("any.txt") + if err != nil { + t.Errorf("Expected no error for default mock, got %v", err) + } + if file != nil { + t.Errorf("Expected nil file for default mock, got %v", file) + } +} + +func TestFileSystemInterface(_ *testing.T) { + // Test that OSFileSystem implements FileSystem interface + var _ FileSystem = OSFileSystem{} + + // Test that MockFS implements FileSystem interface + var _ FileSystem = MockFS{} +} diff --git a/internal/sysmonitor/memory.go b/internal/sysmonitor/memory.go new file mode 100644 index 0000000..a40ff0b --- /dev/null +++ b/internal/sysmonitor/memory.go @@ -0,0 +1,16 @@ +package sysmonitor + +// SystemMemory represents system memory information in bytes. +type SystemMemory struct { + Total uint64 + Available uint64 +} + +type ProcessMemoryReader interface { + Sample() (SystemMemory, error) +} + +// NewProcessMemoryReader creates a new process memory reader. +func NewProcessMemoryReader(fs FileSystem) ProcessMemoryReader { + return newPlatformMemoryReader(fs) +} diff --git a/internal/sysmonitor/memory_darwin.go b/internal/sysmonitor/memory_darwin.go new file mode 100644 index 0000000..b91251a --- /dev/null +++ b/internal/sysmonitor/memory_darwin.go @@ -0,0 +1,72 @@ +//go:build darwin && cgo + +package sysmonitor + +/* +#include +#include +#include + +kern_return_t get_vm_stats(vm_statistics64_t vmstat) { + mach_msg_type_number_t count = HOST_VM_INFO64_COUNT; + mach_port_t host_port = mach_host_self(); + kern_return_t ret = host_statistics64(host_port, HOST_VM_INFO64, (host_info64_t)vmstat, &count); + + // Important: mach_host_self() returns a port send right that must be deallocated, + // unlike mach_task_self() which usually doesn't. + // However, standard practice often keeps host_port cached. + // Given the snippet, we'll ensure we don't leak the port reference if possible, + // though mach_host_self implementation details vary. + mach_port_deallocate(mach_task_self(), host_port); + + return ret; +} + +uint64_t get_hw_memsize() { + uint64_t memsize = 0; + size_t len = sizeof(memsize); + int mib[2] = {CTL_HW, HW_MEMSIZE}; + sysctl(mib, 2, &memsize, &len, NULL, 0); + return memsize; +} +*/ +import "C" + +import ( + "fmt" +) + +type darwinMemoryReader struct{} + +// newPlatformMemoryReader is the factory entry point. +func newPlatformMemoryReader(_ FileSystem) ProcessMemoryReader { + return &darwinMemoryReader{} +} + +// Sample returns the current system memory statistics. +func (d *darwinMemoryReader) Sample() (SystemMemory, error) { + total := uint64(C.get_hw_memsize()) + if total == 0 { + return SystemMemory{}, fmt.Errorf("failed to get total memory via sysctl") + } + + var vmStat C.vm_statistics64_data_t + if ret := C.get_vm_stats(&vmStat); ret != C.KERN_SUCCESS { + return SystemMemory{}, fmt.Errorf("failed to get host VM statistics: kern_return_t=%d", ret) + } + + // Page size is usually 16384 (16kb) on M1/M2/M3 (ARM64) and 4096 (4kb) on Intel. + // C.vm_kernel_page_size handles this automatically. + pageSize := uint64(C.vm_kernel_page_size) + + free := uint64(vmStat.free_count) * pageSize + inactive := uint64(vmStat.inactive_count) * pageSize + speculative := uint64(vmStat.speculative_count) * pageSize + + // "Available" memory on macOS is generally considered to be: + // Free + Inactive (file cache that can be dropped) + Speculative + return SystemMemory{ + Total: total, + Available: free + inactive + speculative, + }, nil +} diff --git a/internal/sysmonitor/memory_darwin_test.go b/internal/sysmonitor/memory_darwin_test.go new file mode 100644 index 0000000..3ad2157 --- /dev/null +++ b/internal/sysmonitor/memory_darwin_test.go @@ -0,0 +1,60 @@ +//go:build darwin + +package sysmonitor + +import ( + "testing" +) + +func TestDarwinMemoryReaderIntegration(t *testing.T) { + reader := newPlatformMemoryReader(nil).(*darwinMemoryReader) + + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Failed to sample Darwin memory: %v", err) + } + + if mem.Total == 0 { + t.Error("Expected non-zero total memory") + } + + if mem.Available == 0 { + t.Error("Expected non-zero available memory") + } + + if mem.Available > mem.Total { + t.Errorf("Available memory (%d) should not exceed total memory (%d)", mem.Available, mem.Total) + } + + t.Logf("Darwin Memory - Total: %d bytes, Available: %d bytes", mem.Total, mem.Available) +} + +func TestNewProcessMemoryReader(t *testing.T) { + // Test the public API function + reader := NewProcessMemoryReader(nil) + + // Should return a non-nil ProcessMemoryReader + if reader == nil { + t.Fatal("NewProcessMemoryReader returned nil") + } +} + +func TestDarwinMemoryReader_ReasonableValues(t *testing.T) { + reader := newPlatformMemoryReader(nil).(*darwinMemoryReader) + + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Failed to sample Darwin memory: %v", err) + } + + // Darwin systems typically have at least 1GB RAM + const minExpectedMemory = 1024 * 1024 * 1024 // 1GB in bytes + if mem.Total < minExpectedMemory { + t.Errorf("Total memory (%d bytes) seems unreasonably low, expected at least %d bytes", mem.Total, minExpectedMemory) + } + + // Available memory should be less than total + if mem.Available >= mem.Total { + t.Errorf("Available memory (%d) should be less than total memory (%d)", mem.Available, mem.Total) + } +} diff --git a/internal/sysmonitor/memory_fallback.go b/internal/sysmonitor/memory_fallback.go new file mode 100644 index 0000000..449db28 --- /dev/null +++ b/internal/sysmonitor/memory_fallback.go @@ -0,0 +1,25 @@ +//go:build !linux && !windows && (!darwin || !cgo) + +package sysmonitor + +import ( + "errors" + "runtime" +) + +// fallbackMemoryReader is a placeholder for unsupported platforms. +type fallbackMemoryReader struct{} + +// newPlatformMemoryReader is the factory entry point. +func newPlatformMemoryReader(_ FileSystem) ProcessMemoryReader { + return &fallbackMemoryReader{} +} + +// Sample returns an error indicating that memory monitoring is not supported. +func (f *fallbackMemoryReader) Sample() (SystemMemory, error) { + if runtime.GOOS == "darwin" { + // Darwin memory monitoring is not supported on platforms that don't have CGO enabled. + return SystemMemory{}, errors.New("memory monitoring on Darwin requires CGO_ENABLED=1") + } + return SystemMemory{}, errors.New("memory monitoring not supported on this platform") +} diff --git a/internal/sysmonitor/memory_fallback_test.go b/internal/sysmonitor/memory_fallback_test.go new file mode 100644 index 0000000..a8c5ea3 --- /dev/null +++ b/internal/sysmonitor/memory_fallback_test.go @@ -0,0 +1,74 @@ +//go:build !linux && !windows && (!darwin || !cgo) + +package sysmonitor + +import ( + "errors" + "runtime" + "strings" + "testing" +) + +func TestNewProcessMemoryReader(t *testing.T) { + // Test the public API function + reader := NewProcessMemoryReader(nil) + + // Should return a fallbackMemoryReader + if reader == nil { + t.Fatal("NewProcessMemoryReader returned nil") + } +} + +func TestFallbackMemoryReader(t *testing.T) { + reader := newPlatformMemoryReader(nil).(*fallbackMemoryReader) + + _, err := reader.Sample() + if err == nil { + t.Fatal("Expected error from fallback memory reader") + } + + expectedMsg := "memory monitoring not supported on this platform" + if runtime.GOOS == "darwin" { + expectedMsg = "memory monitoring on Darwin requires CGO_ENABLED=1" + } + + if !strings.Contains(err.Error(), expectedMsg) { + t.Errorf("Expected error message to contain %q, got %q", expectedMsg, err.Error()) + } +} + +func TestFallbackMemoryReader_Darwin(t *testing.T) { + if runtime.GOOS != "darwin" { + t.Skip("This test is only relevant on Darwin") + } + + reader := newPlatformMemoryReader(nil).(*fallbackMemoryReader) + + _, err := reader.Sample() + if err == nil { + t.Fatal("Expected error from fallback memory reader on Darwin") + } + + expectedMsg := "memory monitoring on Darwin requires CGO_ENABLED=1" + if !errors.Is(err, err) || !strings.Contains(err.Error(), expectedMsg) { + t.Errorf("Expected error message to contain %q, got %q", expectedMsg, err.Error()) + } +} + +func TestFallbackMemoryReader_OtherPlatforms(t *testing.T) { + if runtime.GOOS == "darwin" { + t.Skip("This test is only relevant on non-Darwin platforms") + } + + reader := newPlatformMemoryReader(nil).(*fallbackMemoryReader) + + _, err := reader.Sample() + if err == nil { + t.Fatal("Expected error from fallback memory reader") + } + + expectedMsg := "memory monitoring not supported on this platform" + if !strings.Contains(err.Error(), expectedMsg) { + t.Errorf("Expected error message to contain %q, got %q", expectedMsg, err.Error()) + } +} diff --git a/internal/sysmonitor/memory_linux.go b/internal/sysmonitor/memory_linux.go new file mode 100644 index 0000000..38e1c65 --- /dev/null +++ b/internal/sysmonitor/memory_linux.go @@ -0,0 +1,314 @@ +//go:build linux + +package sysmonitor + +import ( + "bufio" + "bytes" + "fmt" + "io" + "strconv" + "strings" + "sync" +) + +// linuxMemoryReader implements ProcessMemoryReader for Linux. +// It encapsulates the logic for auto-detecting Cgroup/Host memory stats. +type linuxMemoryReader struct { + // delegate is the active strategy (CgroupV2, CgroupV1, or Host). + delegate func() (SystemMemory, error) + fs FileSystem + mu sync.RWMutex + once sync.Once +} + +// newPlatformMemoryReader is the factory entry point used by memory.go. +func newPlatformMemoryReader(fs FileSystem) ProcessMemoryReader { + return &linuxMemoryReader{ + fs: fs, + } +} + +// Sample returns the current system memory statistics. +// It lazily initializes the correct strategy (Cgroup vs Host) on the first call. +func (m *linuxMemoryReader) Sample() (SystemMemory, error) { + m.once.Do(func() { + m.detectEnvironment() + }) + + m.mu.RLock() + strategy := m.delegate + m.mu.RUnlock() + + if strategy == nil { + return SystemMemory{}, fmt.Errorf("failed to initialize memory reader strategy") + } + + return strategy() +} + +// detectEnvironment checks for Cgroups and sets the appropriate delegate strategy. +func (m *linuxMemoryReader) detectEnvironment() { + // 1. Try Cgroup V2 + if m.cgroupFilesExist(cgroupV2Config) { + m.mu.Lock() + m.delegate = m.makeCgroupV2Strategy() + m.mu.Unlock() + return + } + + // 2. Try Cgroup V1 + if m.cgroupFilesExist(cgroupV1Config) { + m.mu.Lock() + m.delegate = m.makeCgroupV1Strategy() + m.mu.Unlock() + return + } + + // 3. Fallback to Host /proc/meminfo + m.mu.Lock() + m.delegate = m.readHostMemory + m.mu.Unlock() +} + +// readHostMemory reads memory info from /proc/meminfo (via FS abstraction) +func (m *linuxMemoryReader) readHostMemory() (SystemMemory, error) { + file, err := m.fs.Open("/proc/meminfo") + if err != nil { + return SystemMemory{}, fmt.Errorf("failed to open /proc/meminfo: %w", err) + } + defer file.Close() + + return parseMemInfo(file) +} + +// makeCgroupV2Strategy returns a function bound to the current FS instance +func (m *linuxMemoryReader) makeCgroupV2Strategy() func() (SystemMemory, error) { + return func() (SystemMemory, error) { + return readCgroupMemoryWithFS(m.fs, cgroupV2Config) + } +} + +// makeCgroupV1Strategy returns a function bound to the current FS instance +func (m *linuxMemoryReader) makeCgroupV1Strategy() func() (SystemMemory, error) { + return func() (SystemMemory, error) { + return readCgroupMemoryWithFS(m.fs, cgroupV1Config) + } +} + +// cgroupFilesExist checks if the required cgroup files exist and are readable +func (m *linuxMemoryReader) cgroupFilesExist(config cgroupMemoryConfig) bool { + // Check if all required files can be read + paths := []string{config.usagePath, config.limitPath, config.statPath} + for _, path := range paths { + if _, err := m.fs.ReadFile(path); err != nil { + return false + } + } + return true +} + +type cgroupMemoryConfig struct { + usagePath string + limitPath string + statPath string + statKey string + version string + checkUnlimited bool +} + +var ( + cgroupV2Config = cgroupMemoryConfig{ + usagePath: "/sys/fs/cgroup/memory.current", + limitPath: "/sys/fs/cgroup/memory.max", + statPath: "/sys/fs/cgroup/memory.stat", + statKey: "inactive_file", + version: "v2", + checkUnlimited: false, + } + cgroupV1Config = cgroupMemoryConfig{ + usagePath: "/sys/fs/cgroup/memory/memory.usage_in_bytes", + limitPath: "/sys/fs/cgroup/memory/memory.limit_in_bytes", + statPath: "/sys/fs/cgroup/memory/memory.stat", + statKey: "total_inactive_file", + version: "v1", + checkUnlimited: true, + } +) + +func readCgroupMemoryWithFS(fs FileSystem, config cgroupMemoryConfig) (SystemMemory, error) { + usage, err := readCgroupValueWithFS(fs, config.usagePath, false) + if err != nil { + return SystemMemory{}, fmt.Errorf("failed to read cgroup %s memory usage: %w", config.version, err) + } + + limit, err := readCgroupValueWithFS(fs, config.limitPath, config.checkUnlimited) + if err != nil { + return SystemMemory{}, fmt.Errorf("failed to read cgroup %s memory limit: %w", config.version, err) + } + + // Parse memory.stat to find reclaimable memory + inactiveFile, err := readCgroupStatWithFS(fs, config.statPath, config.statKey) + if err != nil { + inactiveFile = 0 // Default to 0 if unavailable + } + + // Available = (Limit - Usage) + Reclaimable + var available uint64 + if usage > limit { + available = inactiveFile // Only reclaimable memory is available + } else { + available = (limit - usage) + inactiveFile + } + + if available > limit { + available = limit + } + + return SystemMemory{ + Total: limit, + Available: available, + }, nil +} + +func readCgroupValueWithFS(fs FileSystem, path string, checkUnlimited bool) (uint64, error) { + data, err := fs.ReadFile(path) + if err != nil { + return 0, fmt.Errorf("failed to read file %s: %w", path, err) + } + str := strings.TrimSpace(string(data)) + if str == "max" { + return 0, fmt.Errorf("unlimited memory limit") + } + val, err := strconv.ParseUint(str, 10, 64) + if err != nil { + return 0, fmt.Errorf("failed to parse value %q from %s: %w", str, path, err) + } + // Check for "unlimited" (random huge number in V1) + if checkUnlimited && val > (1<<60) { + return 0, fmt.Errorf("unlimited memory limit") + } + return val, nil +} + +func readCgroupStatWithFS(fs FileSystem, path string, key string) (uint64, error) { + f, err := fs.Open(path) + if err != nil { + return 0, fmt.Errorf("failed to open file %s: %w", path, err) + } + defer f.Close() + + scanner := bufio.NewScanner(f) + for scanner.Scan() { + line := scanner.Bytes() + if bytes.HasPrefix(line, []byte(key)) { + fields := bytes.Fields(line) + if len(fields) >= 2 { + val, err := strconv.ParseUint(string(fields[1]), 10, 64) + if err != nil { + return 0, fmt.Errorf("failed to parse value for key %q in %s: %w", key, path, err) + } + return val, nil + } + } + } + if err := scanner.Err(); err != nil { + return 0, fmt.Errorf("error reading %s: %w", path, err) + } + return 0, fmt.Errorf("key %q not found in %s", key, path) +} + +func parseMemInfo(r io.Reader) (SystemMemory, error) { + memFields, err := parseMemInfoFields(r) + if err != nil { + return SystemMemory{}, err + } + + if memFields.total == 0 { + return SystemMemory{}, fmt.Errorf("could not find MemTotal in /proc/meminfo") + } + + available, err := calculateAvailableMemory(memFields) + if err != nil { + return SystemMemory{}, err + } + + if available > memFields.total { + available = memFields.total + } + + return SystemMemory{ + Total: memFields.total, + Available: available, + }, nil +} + +type memInfoFields struct { + total uint64 + available uint64 + free uint64 + cached uint64 + memAvailableFound bool +} + +func parseMemInfoFields(r io.Reader) (memInfoFields, error) { + scanner := bufio.NewScanner(r) + var fields memInfoFields + + for scanner.Scan() { + line := scanner.Text() + lineFields := strings.Fields(line) + + if len(lineFields) < 2 { + continue + } + + key := strings.TrimSuffix(lineFields[0], ":") + value, err := strconv.ParseUint(lineFields[1], 10, 64) + if err != nil { + continue + } + + const maxValueBeforeOverflow = (1<<64 - 1) / 1024 + if value > maxValueBeforeOverflow { + return memInfoFields{}, fmt.Errorf( + "memory value too large: %d kB would overflow when converting to bytes", value) + } + value *= 1024 + + switch key { + case "MemTotal": + fields.total = value + case "MemAvailable": + fields.available = value + fields.memAvailableFound = true + case "MemFree": + fields.free = value + case "Cached": + fields.cached = value + } + + if fields.total > 0 && fields.memAvailableFound { + break + } + } + + if err := scanner.Err(); err != nil { + return memInfoFields{}, fmt.Errorf("error reading meminfo: %w", err) + } + + return fields, nil +} + +func calculateAvailableMemory(fields memInfoFields) (uint64, error) { + if fields.memAvailableFound { + return fields.available, nil + } + available := fields.free + fields.cached + if available == 0 { + return 0, fmt.Errorf( + "could not find MemAvailable in /proc/meminfo and fallback calculation failed " + + "(MemFree and Cached not found or both zero)") + } + return available, nil +} diff --git a/internal/sysmonitor/memory_linux_test.go b/internal/sysmonitor/memory_linux_test.go new file mode 100644 index 0000000..593f942 --- /dev/null +++ b/internal/sysmonitor/memory_linux_test.go @@ -0,0 +1,578 @@ +//go:build linux + +package sysmonitor + +import ( + "bytes" + "errors" + "fmt" + "strings" + "testing" +) + +func TestLinuxMemoryReader_CgroupV2(t *testing.T) { + fs := &MockFileSystem{ + Files: map[string][]byte{ + "/sys/fs/cgroup/memory.current": []byte("123456789"), + "/sys/fs/cgroup/memory.max": []byte("987654321"), + "/sys/fs/cgroup/memory.stat": []byte("inactive_file 111111\nother 222"), + }, + } + + reader := newPlatformMemoryReader(fs).(*linuxMemoryReader) + + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Expected success, got error: %v", err) + } + + if mem.Total != 987654321 { + t.Errorf("Expected total 987654321, got %d", mem.Total) + } + expectedAvail := uint64((987654321 - 123456789) + 111111) + if mem.Available != expectedAvail { + t.Errorf("Expected available %d, got %d", expectedAvail, mem.Available) + } +} + +func TestLinuxMemoryReader_CgroupV1(t *testing.T) { + fs := &MockFileSystem{ + Files: map[string][]byte{ + "/sys/fs/cgroup/memory/memory.usage_in_bytes": []byte("222222222"), + "/sys/fs/cgroup/memory/memory.limit_in_bytes": []byte("888888888"), + "/sys/fs/cgroup/memory/memory.stat": []byte("total_inactive_file 333333\n"), + }, + } + + reader := newPlatformMemoryReader(fs).(*linuxMemoryReader) + + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Expected success, got error: %v", err) + } + + expectedAvail := uint64((888888888 - 222222222) + 333333) + if mem.Total != 888888888 || mem.Available != expectedAvail { + t.Errorf("Expected {Total:888888888, Available:%d}, got %+v", expectedAvail, mem) + } +} + +func TestLinuxMemoryReader_CgroupV1_Oversubscribed(t *testing.T) { + // Test case where usage > limit (oversubscribed memory) + fs := &MockFileSystem{ + Files: map[string][]byte{ + "/sys/fs/cgroup/memory/memory.usage_in_bytes": []byte("900000000"), // Usage exceeds limit + "/sys/fs/cgroup/memory/memory.limit_in_bytes": []byte("800000000"), + "/sys/fs/cgroup/memory/memory.stat": []byte("total_inactive_file 50000000\n"), + }, + } + + reader := newPlatformMemoryReader(fs).(*linuxMemoryReader) + + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Expected success, got error: %v", err) + } + + // When usage > limit, available should equal inactive_file only + expectedAvail := uint64(50000000) + if mem.Total != 800000000 { + t.Errorf("Expected total 800000000, got %d", mem.Total) + } + if mem.Available != expectedAvail { + t.Errorf("Expected available %d (inactive_file only), got %d", expectedAvail, mem.Available) + } +} + +func TestLinuxMemoryReader_CgroupV1_AvailableCapped(t *testing.T) { + // Test case where calculated available exceeds limit and gets capped + fs := &MockFileSystem{ + Files: map[string][]byte{ + "/sys/fs/cgroup/memory/memory.usage_in_bytes": []byte("100000000"), // Low usage + "/sys/fs/cgroup/memory/memory.limit_in_bytes": []byte("500000000"), + "/sys/fs/cgroup/memory/memory.stat": []byte("total_inactive_file 600000000\n"), // Very high inactive_file + }, + } + + reader := newPlatformMemoryReader(fs).(*linuxMemoryReader) + + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Expected success, got error: %v", err) + } + + // Calculation: (500000000 - 100000000) + 600000000 = 400000000 + 600000000 = 1,000,000,000 + // But this exceeds limit (500000000), so available should be capped at limit + expectedAvail := uint64(500000000) + if mem.Total != 500000000 { + t.Errorf("Expected total 500000000, got %d", mem.Total) + } + if mem.Available != expectedAvail { + t.Errorf("Expected available %d (capped at limit), got %d", expectedAvail, mem.Available) + } +} + +func TestLinuxMemoryReader_HostFallback(t *testing.T) { + fs := &MockFileSystem{ + Files: map[string][]byte{ + "/proc/meminfo": []byte(`MemTotal: 16384000 kB +MemFree: 8192000 kB +Cached: 4096000 kB +MemAvailable: 10000000 kB +`), + }, + OpenErrs: map[string]error{ + "/sys/fs/cgroup/memory.current": errors.New("no cgroup v2"), + "/sys/fs/cgroup/memory/memory.usage_in_bytes": errors.New("no cgroup v1"), + }, + } + + reader := newPlatformMemoryReader(fs).(*linuxMemoryReader) + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Expected success: %v", err) + } + + if mem.Total != 16384000*1024 || mem.Available != 10000000*1024 { + t.Errorf("Expected MemAvailable parsing, got Total:%d Avail:%d", mem.Total, mem.Available) + } +} + +func TestLinuxMemoryReader_MemInfoFallback(t *testing.T) { + fs := &MockFileSystem{ + Files: map[string][]byte{ + "/proc/meminfo": []byte(`MemTotal: 16384000 kB +MemFree: 8192000 kB +Cached: 4096000 kB +`), + }, + } + + reader := newPlatformMemoryReader(fs).(*linuxMemoryReader) + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Expected success: %v", err) + } + + expectedAvail := uint64((8192000 + 4096000) * 1024) + if mem.Available != expectedAvail { + t.Errorf("Expected fallback calculation %d, got %d", expectedAvail, mem.Available) + } +} + +func TestLinuxMemoryReader_OverflowProtection(t *testing.T) { + // Use a value that definitely exceeds maxValueBeforeOverflow = (1<<64 - 1) / 1024 = 18014398509481983 + _, err := parseMemInfoFields(bytes.NewReader([]byte(`MemTotal: 20000000000000000 kB`))) + if err == nil { + t.Error("Expected overflow error") + } +} + +func TestCgroupDetection(t *testing.T) { + // Test that cgroup V1 detection works with our setup + fs := &MockFileSystem{ + Files: map[string][]byte{ + "/sys/fs/cgroup/memory/memory.usage_in_bytes": []byte("1000000"), + "/sys/fs/cgroup/memory/memory.limit_in_bytes": []byte("18446744073709551615"), + "/sys/fs/cgroup/memory/memory.stat": []byte("total_inactive_file 500000\n"), + }, + } + + reader := newPlatformMemoryReader(fs).(*linuxMemoryReader) + + // Call detectEnvironment manually to see what happens + reader.detectEnvironment() + + // Check if delegate was set (should be set for cgroup V1) + reader.mu.RLock() + delegateSet := reader.delegate != nil + reader.mu.RUnlock() + + if !delegateSet { + t.Error("Cgroup V1 detection should have succeeded") + } +} + +func TestNewProcessMemoryReader(t *testing.T) { + reader := NewProcessMemoryReader(&MockFileSystem{}) + + // Should return a linuxMemoryReader (or platform-specific implementation) + if reader == nil { + t.Fatal("NewProcessMemoryReader returned nil") + } +} + +func TestLinuxMemoryReader_ErrorPaths(t *testing.T) { + tests := []struct { + name string + setup func(*MockFileSystem) + err string + }{ + { + "MemTotal missing", + func(fs *MockFileSystem) { + if fs.Files == nil { + fs.Files = make(map[string][]byte) + } + fs.Files["/proc/meminfo"] = []byte("MemFree: 1000 kB\n") + }, + "could not find MemTotal", + }, + { + "Meminfo parse fail", + func(fs *MockFileSystem) { + if fs.OpenErrs == nil { + fs.OpenErrs = make(map[string]error) + } + fs.OpenErrs["/proc/meminfo"] = errors.New("no meminfo") + }, + "failed to open /proc/meminfo", + }, + { + "Cgroup unlimited V1", + func(fs *MockFileSystem) { + // Replace the Files map entirely to ensure clean setup + fs.Files = map[string][]byte{ + "/sys/fs/cgroup/memory/memory.usage_in_bytes": []byte("1000000"), + "/sys/fs/cgroup/memory/memory.limit_in_bytes": []byte("18446744073709551615"), + "/sys/fs/cgroup/memory/memory.stat": []byte("total_inactive_file 500000\n"), + } + // Block Cgroup V2 detection by making V2 files return errors + fs.OpenErrs = map[string]error{ + "/sys/fs/cgroup/memory.current": errors.New("no cgroup v2"), + "/sys/fs/cgroup/memory.max": errors.New("no cgroup v2"), + "/proc/meminfo": errors.New("cgroup should be used"), + } + }, + "unlimited memory limit", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + fs := &MockFileSystem{} + tt.setup(fs) + + // Verify mock setup before creating reader + if tt.name == "Cgroup unlimited V1" { + if _, err := fs.ReadFile("/sys/fs/cgroup/memory/memory.limit_in_bytes"); err != nil { + t.Fatalf("Mock setup failed: %v", err) + } + } + + reader := newPlatformMemoryReader(fs).(*linuxMemoryReader) + + // Force detection (your once.Do will trigger it) + reader.once.Do(func() { reader.detectEnvironment() }) + + _, err := reader.Sample() + if err == nil || !strings.Contains(err.Error(), tt.err) { + t.Errorf("Expected error containing %q, got %v", tt.err, err) + } + }) + } +} + +// Helper function for cgroup value tests +func runCgroupValueTest( + t *testing.T, + name, fileContent string, + checkUnlimited bool, + expected uint64, + expectError bool, + errorContains string, +) { + fs := &MockFileSystem{} + if name != "File read error" { + fs.Files = map[string][]byte{ + "/test/file": []byte(fileContent), + } + } else { + fs.OpenErrs = map[string]error{ + "/test/file": fmt.Errorf("permission denied"), + } + } + + result, err := readCgroupValueWithFS(fs, "/test/file", checkUnlimited) + + if expectError { + if err == nil { + t.Errorf("Expected error containing %q, got nil", errorContains) + } else if !strings.Contains(err.Error(), errorContains) { + t.Errorf("Expected error containing %q, got %v", errorContains, err) + } + } else { + if err != nil { + t.Errorf("Expected no error, got %v", err) + } + if result != expected { + t.Errorf("Expected %d, got %d", expected, result) + } + } +} + +// Helper function for cgroup stat tests +func runCgroupStatTest( + t *testing.T, + name, fileContent, key string, + expected uint64, + expectError bool, + errorContains string, +) { + fs := &MockFileSystem{} + if name != "File open error" { + fs.Files = map[string][]byte{ + "/test/stat": []byte(fileContent), + } + } else { + fs.OpenErrs = map[string]error{ + "/test/stat": fmt.Errorf("permission denied"), + } + } + + result, err := readCgroupStatWithFS(fs, "/test/stat", key) + + if expectError { + if err == nil { + t.Errorf("Expected error containing %q, got nil", errorContains) + } else if !strings.Contains(err.Error(), errorContains) { + t.Errorf("Expected error containing %q, got %v", errorContains, err) + } + } else { + if err != nil { + t.Errorf("Expected no error, got %v", err) + } + if result != expected { + t.Errorf("Expected %d, got %d", expected, result) + } + } +} + +// Test readCgroupValueWithFS edge cases +func TestReadCgroupValueWithFS(t *testing.T) { + tests := []struct { + name string + fileContent string + checkUnlimited bool + expected uint64 + expectError bool + errorContains string + }{ + { + name: "Normal value", + fileContent: "1024000", + checkUnlimited: false, + expected: 1024000, + expectError: false, + }, + { + name: "Unlimited max value", + fileContent: "max", + checkUnlimited: true, + expected: 0, + expectError: true, + errorContains: "unlimited memory limit", + }, + { + name: "Unlimited large value", + fileContent: "18446744073709551615", // ^uint64(0) + checkUnlimited: true, + expected: 0, + expectError: true, + errorContains: "unlimited memory limit", + }, + { + name: "Large value but checkUnlimited false", + fileContent: "18446744073709551615", + checkUnlimited: false, + expected: 18446744073709551615, + expectError: false, + }, + { + name: "Invalid number", + fileContent: "invalid", + checkUnlimited: false, + expected: 0, + expectError: true, + errorContains: "failed to parse value", + }, + { + name: "Empty file", + fileContent: "", + checkUnlimited: false, + expected: 0, + expectError: true, + errorContains: "failed to parse value", + }, + { + name: "Whitespace padded", + fileContent: " 12345 \n", + checkUnlimited: false, + expected: 12345, + expectError: false, + }, + { + name: "File read error", + fileContent: "", + checkUnlimited: false, + expected: 0, + expectError: true, + errorContains: "failed to read file", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + runCgroupValueTest(t, tt.name, tt.fileContent, tt.checkUnlimited, tt.expected, tt.expectError, tt.errorContains) + }) + } +} + +// Test readCgroupStatWithFS edge cases +func TestReadCgroupStatWithFS(t *testing.T) { + tests := []struct { + name string + fileContent string + key string + expected uint64 + expectError bool + errorContains string + }{ + { + name: "Normal key found", + fileContent: "total_inactive_file 50000\nother_key 1000\n", + key: "total_inactive_file", + expected: 50000, + expectError: false, + }, + { + name: "Key at end of file", + fileContent: "first_key 100\ntotal_inactive_file 75000", + key: "total_inactive_file", + expected: 75000, + expectError: false, + }, + { + name: "Key not found", + fileContent: "other_key 100\nanother_key 200\n", + key: "total_inactive_file", + expected: 0, + expectError: true, + errorContains: "key \"total_inactive_file\" not found", + }, + { + name: "Invalid value", + fileContent: "total_inactive_file invalid\n", + key: "total_inactive_file", + expected: 0, + expectError: true, + errorContains: "failed to parse value", + }, + { + name: "Key without value", + fileContent: "total_inactive_file\nother_key 100\n", + key: "total_inactive_file", + expected: 0, + expectError: true, + errorContains: "key \"total_inactive_file\" not found", + }, + { + name: "File open error", + fileContent: "", + key: "total_inactive_file", + expected: 0, + expectError: true, + errorContains: "failed to open file", + }, + { + name: "Multiple spaces", + fileContent: "total_inactive_file 12345\n", + key: "total_inactive_file", + expected: 12345, + expectError: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + runCgroupStatTest(t, tt.name, tt.fileContent, tt.key, tt.expected, tt.expectError, tt.errorContains) + }) + } +} + +// Test parseMemInfo edge cases +func TestParseMemInfo(t *testing.T) { + tests := []struct { + name string + meminfo string + expectedTotal uint64 + expectedAvail uint64 + expectError bool + errorContains string + }{ + { + name: "Missing MemTotal", + meminfo: "MemFree: 1000 kB\nCached: 2000 kB\n", + expectedTotal: 0, + expectedAvail: 0, + expectError: true, + errorContains: "could not find MemTotal", + }, + { + name: "Invalid MemTotal", + meminfo: "MemTotal: invalid kB\nMemFree: 1000 kB\n", + expectedTotal: 0, + expectedAvail: 0, + expectError: true, + errorContains: "could not find MemTotal", // parseMemInfoFields skips invalid lines + }, + { + name: "Normal case with MemAvailable", + meminfo: "MemTotal: 8000 kB\nMemFree: 1000 kB\nMemAvailable: 6000 kB\nCached: 2000 kB\n", + expectedTotal: 8192000, // 8000 kB * 1024 = 8192000 bytes + expectedAvail: 6144000, // 6000 kB * 1024 = 6144000 bytes + expectError: false, + }, + { + name: "Fallback calculation without MemAvailable", + meminfo: "MemTotal: 8000 kB\nMemFree: 1000 kB\nCached: 2000 kB\n", + expectedTotal: 8192000, // 8000 kB * 1024 = 8192000 bytes + expectedAvail: 0, // Will be calculated by calculateAvailableMemory + expectError: false, + }, + { + name: "Fallback fails when MemFree and Cached are zero", + meminfo: "MemTotal: 8000 kB\nMemFree: 0 kB\nCached: 0 kB\n", + expectedTotal: 0, + expectedAvail: 0, + expectError: true, + errorContains: "fallback calculation failed", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + reader := strings.NewReader(tt.meminfo) + result, err := parseMemInfo(reader) + + if tt.expectError { + if err == nil { + t.Errorf("Expected error containing %q, got nil", tt.errorContains) + } else if !strings.Contains(err.Error(), tt.errorContains) { + t.Errorf("Expected error containing %q, got %v", tt.errorContains, err) + } + } else { + if err != nil { + t.Errorf("Expected no error, got %v", err) + } + if result.Total != tt.expectedTotal { + t.Errorf("Expected total %d, got %d", tt.expectedTotal, result.Total) + } + // Note: Available memory calculation may vary, so we just check it's reasonable + if tt.expectedAvail > 0 && result.Available == 0 { + t.Errorf("Expected some available memory, got 0") + } + } + }) + } +} diff --git a/internal/sysmonitor/memory_windows.go b/internal/sysmonitor/memory_windows.go new file mode 100644 index 0000000..008da3a --- /dev/null +++ b/internal/sysmonitor/memory_windows.go @@ -0,0 +1,54 @@ +//go:build windows + +package sysmonitor + +import ( + "fmt" + "syscall" + "unsafe" +) + +var ( + kernel32 = syscall.NewLazyDLL("kernel32.dll") + procGlobalMemoryStatusEx = kernel32.NewProc("GlobalMemoryStatusEx") +) + +// memoryStatusEx matches the MEMORYSTATUSEX structure in Windows API +type memoryStatusEx struct { + dwLength uint32 + dwMemoryLoad uint32 + ullTotalPhys uint64 + ullAvailPhys uint64 + ullTotalPageFile uint64 + ullAvailPageFile uint64 + ullTotalVirtual uint64 + ullAvailVirtual uint64 + ullAvailExtendedVirtual uint64 +} + +// windowsMemoryReader implements ProcessMemoryReader for Windows using Win32 API. +type windowsMemoryReader struct{} + +// newPlatformMemoryReader is the factory entry point. +func newPlatformMemoryReader(_ FileSystem) ProcessMemoryReader { + return &windowsMemoryReader{} +} + +// Sample returns the current system memory statistics. +func (w *windowsMemoryReader) Sample() (SystemMemory, error) { + var memStatus memoryStatusEx + memStatus.dwLength = uint32(unsafe.Sizeof(memStatus)) + + // Call GlobalMemoryStatusEx + ret, _, err := procGlobalMemoryStatusEx.Call(uintptr(unsafe.Pointer(&memStatus))) + + // If the function fails, the return value is zero. + if ret == 0 { + return SystemMemory{}, fmt.Errorf("failed to get system memory status via GlobalMemoryStatusEx: %w", err) + } + + return SystemMemory{ + Total: memStatus.ullTotalPhys, + Available: memStatus.ullAvailPhys, + }, nil +} diff --git a/internal/sysmonitor/memory_windows_test.go b/internal/sysmonitor/memory_windows_test.go new file mode 100644 index 0000000..dc3253e --- /dev/null +++ b/internal/sysmonitor/memory_windows_test.go @@ -0,0 +1,170 @@ +//go:build windows + +package sysmonitor + +import ( + "math" + "testing" +) + +func TestWindowsMemoryReaderIntegration(t *testing.T) { + reader := newPlatformMemoryReader(nil).(*windowsMemoryReader) + + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Failed to sample Windows memory: %v", err) + } + + if mem.Total == 0 { + t.Error("Expected non-zero total memory") + } + + if mem.Available == 0 { + t.Error("Expected non-zero available memory") + } + + if mem.Available > mem.Total { + t.Errorf("Available memory (%d) should not exceed total memory (%d)", mem.Available, mem.Total) + } + + t.Logf("Windows Memory - Total: %d bytes, Available: %d bytes", mem.Total, mem.Available) +} + +func TestNewProcessMemoryReader(t *testing.T) { + // Test the public API function + reader := NewProcessMemoryReader(nil) + + // Should return a windowsMemoryReader + if reader == nil { + t.Fatal("NewProcessMemoryReader returned nil") + } +} + +func TestWindowsMemoryReader_ReasonableValues(t *testing.T) { + reader := newPlatformMemoryReader(nil).(*windowsMemoryReader) + + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Failed to sample Windows memory: %v", err) + } + + // Windows systems typically have at least 1GB RAM + const minExpectedMemory = 1024 * 1024 * 1024 // 1GB in bytes + if mem.Total < minExpectedMemory { + t.Errorf("Total memory (%d bytes) seems unreasonably low, expected at least %d bytes", mem.Total, minExpectedMemory) + } + + // Available memory should be less than or equal to total + if mem.Available > mem.Total { + t.Errorf("Available memory (%d) should not exceed total memory (%d)", mem.Available, mem.Total) + } +} + +func TestWindowsMemoryReader_Consistency(t *testing.T) { + reader := newPlatformMemoryReader(nil).(*windowsMemoryReader) + + // Take multiple samples to ensure consistency + var samples []SystemMemory + for i := 0; i < 3; i++ { + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Failed to sample Windows memory on iteration %d: %v", i, err) + } + samples = append(samples, mem) + } + + // Total memory should be consistent across samples + for i := 1; i < len(samples); i++ { + if samples[i].Total != samples[0].Total { + t.Errorf("Total memory inconsistent: sample 0: %d, sample %d: %d", + samples[0].Total, i, samples[i].Total) + } + } + + // Available memory should be reasonable (not wildly different) + for i := 1; i < len(samples); i++ { + availableDiff := math.Abs(float64(samples[i].Available) - float64(samples[0].Available)) + if availableDiff > float64(samples[0].Total/4) { + t.Errorf("Available memory changed dramatically: sample 0: %d, sample %d: %d", + samples[0].Available, i, samples[i].Available) + } + } +} + +func TestWindowsMemoryReader_MemoryPressure(t *testing.T) { + reader := newPlatformMemoryReader(nil).(*windowsMemoryReader) + + // Take baseline reading + baseline, err := reader.Sample() + if err != nil { + t.Fatalf("Failed to get baseline memory reading: %v", err) + } + + // Allocate some memory to simulate pressure (this won't actually change system memory much, + // but tests that the reader continues to work) + testData := make([]byte, 10*1024*1024) // 10MB + _ = testData // Prevent optimization + + // Take reading after allocation + afterAlloc, err := reader.Sample() + if err != nil { + t.Fatalf("Failed to get memory reading after allocation: %v", err) + } + + // Memory readings should still be valid + if afterAlloc.Total != baseline.Total { + t.Errorf("Total memory changed unexpectedly: before=%d, after=%d", baseline.Total, afterAlloc.Total) + } + + if afterAlloc.Available > afterAlloc.Total { + t.Errorf("Invalid memory reading: available (%d) > total (%d)", afterAlloc.Available, afterAlloc.Total) + } + + // Clean up + testData = nil +} + +// TestWindowsMemoryReader_MultipleReaders tests multiple readers work independently +func TestWindowsMemoryReader_MultipleReaders(t *testing.T) { + reader1 := newPlatformMemoryReader(nil).(*windowsMemoryReader) + reader2 := newPlatformMemoryReader(nil).(*windowsMemoryReader) + + mem1, err := reader1.Sample() + if err != nil { + t.Fatalf("Reader1 failed: %v", err) + } + + mem2, err := reader2.Sample() + if err != nil { + t.Fatalf("Reader2 failed: %v", err) + } + + // Both readers should return the same system memory info + if mem1.Total != mem2.Total { + t.Errorf("Readers returned different total memory: reader1=%d, reader2=%d", mem1.Total, mem2.Total) + } +} + +// TestWindowsMemoryReader_PercentageCalculation tests memory percentage calculations +func TestWindowsMemoryReader_PercentageCalculation(t *testing.T) { + reader := newPlatformMemoryReader(nil).(*windowsMemoryReader) + + mem, err := reader.Sample() + if err != nil { + t.Fatalf("Failed to sample memory: %v", err) + } + + if mem.Total == 0 { + t.Fatal("Total memory is zero, cannot calculate percentages") + } + + // Calculate usage percentage + used := mem.Total - mem.Available + usagePercent := float64(used) / float64(mem.Total) * 100.0 + + if usagePercent < 0.0 || usagePercent > 100.0 { + t.Errorf("Invalid usage percentage: %f%%", usagePercent) + } + + t.Logf("Memory usage: %d/%d bytes (%.2f%%)", used, mem.Total, usagePercent) +} diff --git a/internal/sysmonitor/test_helpers.go b/internal/sysmonitor/test_helpers.go new file mode 100644 index 0000000..26405f7 --- /dev/null +++ b/internal/sysmonitor/test_helpers.go @@ -0,0 +1,53 @@ +package sysmonitor + +import ( + "errors" + "io" + "io/fs" +) + +// MockFileSystem implements FileSystem for testing +type MockFileSystem struct { + Files map[string][]byte + OpenErrs map[string]error +} + +func (m *MockFileSystem) ReadFile(name string) ([]byte, error) { + if content, ok := m.Files[name]; ok { + return content, nil + } + return nil, errors.New("file not found") +} + +func (m *MockFileSystem) Open(name string) (fs.File, error) { + if err, ok := m.OpenErrs[name]; ok { + return nil, err + } + if content, ok := m.Files[name]; ok { + return &mockFile{content: content, pos: 0}, nil + } + return nil, errors.New("file not found") +} + +// mockFile implements fs.File for testing +type mockFile struct { + content []byte + pos int +} + +func (f *mockFile) Read(p []byte) (n int, err error) { + if f.pos >= len(f.content) { + return 0, io.EOF + } + n = copy(p, f.content[f.pos:]) + f.pos += n + return n, nil +} + +func (f *mockFile) Close() error { + return nil +} + +func (f *mockFile) Stat() (fs.FileInfo, error) { + return nil, errors.New("Stat not implemented") +}