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")
+}