Skip to content

Commit 080a33c

Browse files
ndyakovCopilot
andauthored
fix(pool): pool performance (#3565)
* perf(pool): replace hookManager RWMutex with atomic.Pointer and add predefined state slices - Replace hookManager RWMutex with atomic.Pointer for lock-free reads in hot paths - Add predefined state slices to avoid allocations (validFromInUse, validFromCreatedOrIdle, etc.) - Add Clone() method to PoolHookManager for atomic updates - Update AddPoolHook/RemovePoolHook to use copy-on-write pattern - Update all hookManager access points to use atomic Load() Performance improvements: - Eliminates RWMutex contention in Get/Put/Remove hot paths - Reduces allocations by reusing predefined state slices - Lock-free reads allow better CPU cache utilization * perf(pool): eliminate mutex overhead in state machine hot path The state machine was calling notifyWaiters() on EVERY Get/Put operation, which acquired a mutex even when no waiters were present (the common case). Fix: Use atomic waiterCount to check for waiters BEFORE acquiring mutex. This eliminates mutex contention in the hot path (Get/Put operations). Implementation: - Added atomic.Int32 waiterCount field to ConnStateMachine - Increment when adding waiter, decrement when removing - Check waiterCount atomically before acquiring mutex in notifyWaiters() Performance impact: - Before: mutex lock/unlock on every Get/Put (even with no waiters) - After: lock-free atomic check, only acquire mutex if waiters exist - Expected improvement: ~30-50% for Get/Put operations * perf(pool): use predefined state slices to eliminate allocations in hot path The pool was creating new slice literals on EVERY Get/Put operation: - popIdle(): []ConnState{StateCreated, StateIdle} - putConn(): []ConnState{StateInUse} - CompareAndSwapUsed(): []ConnState{StateIdle} and []ConnState{StateInUse} - MarkUnusableForHandoff(): []ConnState{StateInUse, StateIdle, StateCreated} These allocations were happening millions of times per second in the hot path. Fix: Use predefined global slices defined in conn_state.go: - validFromInUse - validFromCreatedOrIdle - validFromCreatedInUseOrIdle Performance impact: - Before: 4 slice allocations per Get/Put cycle - After: 0 allocations (use predefined slices) - Expected improvement: ~30-40% reduction in allocations and GC pressure * perf(pool): optimize TryTransition to reduce atomic operations Further optimize the hot path by: 1. Remove redundant GetState() call in the loop 2. Only check waiterCount after successful CAS (not before loop) 3. Inline the waiterCount check to avoid notifyWaiters() call overhead This reduces atomic operations from 4-5 per Get/Put to 2-3: - Before: GetState() + CAS + waiterCount.Load() + notifyWaiters mutex check - After: CAS + waiterCount.Load() (only if CAS succeeds) Performance impact: - Eliminates 1-2 atomic operations per Get/Put - Expected improvement: ~10-15% for Get/Put operations * perf(pool): add fast path for Get/Put to match master performance Introduced TryTransitionFast() for the hot path (Get/Put operations): - Single CAS operation (same as master's atomic bool) - No waiter notification overhead - No loop through valid states - No error allocation Hot path flow: 1. popIdle(): Try IDLE → IN_USE (fast), fallback to CREATED → IN_USE 2. putConn(): Try IN_USE → IDLE (fast) This matches master's performance while preserving state machine for: - Background operations (handoff/reauth use UNUSABLE state) - State validation (TryTransition still available) - Waiter notification (AwaitAndTransition for blocking) Performance comparison per Get/Put cycle: - Master: 2 atomic CAS operations - State machine (before): 5 atomic operations (2.5x slower) - State machine (after): 2 atomic CAS operations (same as master!) Expected improvement: Restore to baseline ~11,373 ops/sec * combine cas * fix linter * try faster approach * fast semaphore * better inlining for hot path * fix linter issues * use new semaphore in auth as well * linter should be happy now * add comments * Update internal/pool/conn_state.go Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> * address comment * slight reordering * try to cache time if for non-critical calculation * fix wrong benchmark * add concurrent test * fix benchmark report * add additional expect to check output * comment and variable rename --------- Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
1 parent 07e665f commit 080a33c

File tree

12 files changed

+583
-167
lines changed

12 files changed

+583
-167
lines changed

hset_benchmark_test.go

Lines changed: 113 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@ package redis_test
33
import (
44
"context"
55
"fmt"
6+
"sync"
67
"testing"
78
"time"
89

@@ -100,7 +101,82 @@ func benchmarkHSETOperations(b *testing.B, rdb *redis.Client, ctx context.Contex
100101
avgTimePerOp := b.Elapsed().Nanoseconds() / int64(operations*b.N)
101102
b.ReportMetric(float64(avgTimePerOp), "ns/op")
102103
// report average time in milliseconds from totalTimes
103-
avgTimePerOpMs := totalTimes[0].Milliseconds() / int64(len(totalTimes))
104+
sumTime := time.Duration(0)
105+
for _, t := range totalTimes {
106+
sumTime += t
107+
}
108+
avgTimePerOpMs := sumTime.Milliseconds() / int64(len(totalTimes))
109+
b.ReportMetric(float64(avgTimePerOpMs), "ms")
110+
}
111+
112+
// benchmarkHSETOperationsConcurrent performs the actual HSET benchmark for a given scale
113+
func benchmarkHSETOperationsConcurrent(b *testing.B, rdb *redis.Client, ctx context.Context, operations int) {
114+
hashKey := fmt.Sprintf("benchmark_hash_%d", operations)
115+
116+
b.ResetTimer()
117+
b.StartTimer()
118+
totalTimes := []time.Duration{}
119+
120+
for i := 0; i < b.N; i++ {
121+
b.StopTimer()
122+
// Clean up the hash before each iteration
123+
rdb.Del(ctx, hashKey)
124+
b.StartTimer()
125+
126+
startTime := time.Now()
127+
// Perform the specified number of HSET operations
128+
129+
wg := sync.WaitGroup{}
130+
timesCh := make(chan time.Duration, operations)
131+
errCh := make(chan error, operations)
132+
133+
for j := 0; j < operations; j++ {
134+
wg.Add(1)
135+
go func(j int) {
136+
defer wg.Done()
137+
field := fmt.Sprintf("field_%d", j)
138+
value := fmt.Sprintf("value_%d", j)
139+
140+
err := rdb.HSet(ctx, hashKey, field, value).Err()
141+
if err != nil {
142+
errCh <- err
143+
return
144+
}
145+
timesCh <- time.Since(startTime)
146+
}(j)
147+
}
148+
149+
wg.Wait()
150+
close(timesCh)
151+
close(errCh)
152+
153+
// Check for errors
154+
for err := range errCh {
155+
b.Errorf("HSET operation failed: %v", err)
156+
}
157+
158+
for d := range timesCh {
159+
totalTimes = append(totalTimes, d)
160+
}
161+
}
162+
163+
// Stop the timer to calculate metrics
164+
b.StopTimer()
165+
166+
// Report operations per second
167+
opsPerSec := float64(operations*b.N) / b.Elapsed().Seconds()
168+
b.ReportMetric(opsPerSec, "ops/sec")
169+
170+
// Report average time per operation
171+
avgTimePerOp := b.Elapsed().Nanoseconds() / int64(operations*b.N)
172+
b.ReportMetric(float64(avgTimePerOp), "ns/op")
173+
// report average time in milliseconds from totalTimes
174+
175+
sumTime := time.Duration(0)
176+
for _, t := range totalTimes {
177+
sumTime += t
178+
}
179+
avgTimePerOpMs := sumTime.Milliseconds() / int64(len(totalTimes))
104180
b.ReportMetric(float64(avgTimePerOpMs), "ms")
105181
}
106182

@@ -134,6 +210,37 @@ func BenchmarkHSETPipelined(b *testing.B) {
134210
}
135211
}
136212

213+
func BenchmarkHSET_Concurrent(b *testing.B) {
214+
ctx := context.Background()
215+
216+
// Setup Redis client
217+
rdb := redis.NewClient(&redis.Options{
218+
Addr: "localhost:6379",
219+
DB: 0,
220+
PoolSize: 100,
221+
})
222+
defer rdb.Close()
223+
224+
// Test connection
225+
if err := rdb.Ping(ctx).Err(); err != nil {
226+
b.Skipf("Redis server not available: %v", err)
227+
}
228+
229+
// Clean up before and after tests
230+
defer func() {
231+
rdb.FlushDB(ctx)
232+
}()
233+
234+
// Reduced scales to avoid overwhelming the system with too many concurrent goroutines
235+
scales := []int{1, 10, 100, 1000}
236+
237+
for _, scale := range scales {
238+
b.Run(fmt.Sprintf("HSET_%d_operations_concurrent", scale), func(b *testing.B) {
239+
benchmarkHSETOperationsConcurrent(b, rdb, ctx, scale)
240+
})
241+
}
242+
}
243+
137244
// benchmarkHSETPipelined performs HSET benchmark using pipelining
138245
func benchmarkHSETPipelined(b *testing.B, rdb *redis.Client, ctx context.Context, operations int) {
139246
hashKey := fmt.Sprintf("benchmark_hash_pipelined_%d", operations)
@@ -177,7 +284,11 @@ func benchmarkHSETPipelined(b *testing.B, rdb *redis.Client, ctx context.Context
177284
avgTimePerOp := b.Elapsed().Nanoseconds() / int64(operations*b.N)
178285
b.ReportMetric(float64(avgTimePerOp), "ns/op")
179286
// report average time in milliseconds from totalTimes
180-
avgTimePerOpMs := totalTimes[0].Milliseconds() / int64(len(totalTimes))
287+
sumTime := time.Duration(0)
288+
for _, t := range totalTimes {
289+
sumTime += t
290+
}
291+
avgTimePerOpMs := sumTime.Milliseconds() / int64(len(totalTimes))
181292
b.ReportMetric(float64(avgTimePerOpMs), "ms")
182293
}
183294

internal/auth/streaming/pool_hook.go

Lines changed: 7 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -34,9 +34,10 @@ type ReAuthPoolHook struct {
3434
shouldReAuth map[uint64]func(error)
3535
shouldReAuthLock sync.RWMutex
3636

37-
// workers is a semaphore channel limiting concurrent re-auth operations
37+
// workers is a semaphore limiting concurrent re-auth operations
3838
// Initialized with poolSize tokens to prevent pool exhaustion
39-
workers chan struct{}
39+
// Uses FastSemaphore for consistency and better performance
40+
workers *internal.FastSemaphore
4041

4142
// reAuthTimeout is the maximum time to wait for acquiring a connection for re-auth
4243
reAuthTimeout time.Duration
@@ -59,16 +60,10 @@ type ReAuthPoolHook struct {
5960
// The poolSize parameter is used to initialize the worker semaphore, ensuring that
6061
// re-auth operations don't exhaust the connection pool.
6162
func NewReAuthPoolHook(poolSize int, reAuthTimeout time.Duration) *ReAuthPoolHook {
62-
workers := make(chan struct{}, poolSize)
63-
// Initialize the workers channel with tokens (semaphore pattern)
64-
for i := 0; i < poolSize; i++ {
65-
workers <- struct{}{}
66-
}
67-
6863
return &ReAuthPoolHook{
6964
shouldReAuth: make(map[uint64]func(error)),
7065
scheduledReAuth: make(map[uint64]bool),
71-
workers: workers,
66+
workers: internal.NewFastSemaphore(int32(poolSize)),
7267
reAuthTimeout: reAuthTimeout,
7368
}
7469
}
@@ -162,10 +157,10 @@ func (r *ReAuthPoolHook) OnPut(_ context.Context, conn *pool.Conn) (bool, bool,
162157
r.scheduledLock.Unlock()
163158
r.shouldReAuthLock.Unlock()
164159
go func() {
165-
<-r.workers
160+
r.workers.AcquireBlocking()
166161
// safety first
167162
if conn == nil || (conn != nil && conn.IsClosed()) {
168-
r.workers <- struct{}{}
163+
r.workers.Release()
169164
return
170165
}
171166
defer func() {
@@ -176,7 +171,7 @@ func (r *ReAuthPoolHook) OnPut(_ context.Context, conn *pool.Conn) (bool, bool,
176171
r.scheduledLock.Lock()
177172
delete(r.scheduledReAuth, connID)
178173
r.scheduledLock.Unlock()
179-
r.workers <- struct{}{}
174+
r.workers.Release()
180175
}()
181176

182177
// Create timeout context for connection acquisition

internal/pool/conn.go

Lines changed: 82 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,4 @@
1+
// Package pool implements the pool management
12
package pool
23

34
import (
@@ -17,6 +18,35 @@ import (
1718

1819
var noDeadline = time.Time{}
1920

21+
// Global time cache updated every 50ms by background goroutine.
22+
// This avoids expensive time.Now() syscalls in hot paths like getEffectiveReadTimeout.
23+
// Max staleness: 50ms, which is acceptable for timeout deadline checks (timeouts are typically 3-30 seconds).
24+
var globalTimeCache struct {
25+
nowNs atomic.Int64
26+
}
27+
28+
func init() {
29+
// Initialize immediately
30+
globalTimeCache.nowNs.Store(time.Now().UnixNano())
31+
32+
// Start background updater
33+
go func() {
34+
ticker := time.NewTicker(50 * time.Millisecond)
35+
defer ticker.Stop()
36+
37+
for range ticker.C {
38+
globalTimeCache.nowNs.Store(time.Now().UnixNano())
39+
}
40+
}()
41+
}
42+
43+
// getCachedTimeNs returns the current time in nanoseconds from the global cache.
44+
// This is updated every 50ms by a background goroutine, avoiding expensive syscalls.
45+
// Max staleness: 50ms.
46+
func getCachedTimeNs() int64 {
47+
return globalTimeCache.nowNs.Load()
48+
}
49+
2050
// Global atomic counter for connection IDs
2151
var connIDCounter uint64
2252

@@ -79,6 +109,7 @@ type Conn struct {
79109
expiresAt time.Time
80110

81111
// maintenanceNotifications upgrade support: relaxed timeouts during migrations/failovers
112+
82113
// Using atomic operations for lock-free access to avoid mutex contention
83114
relaxedReadTimeoutNs atomic.Int64 // time.Duration as nanoseconds
84115
relaxedWriteTimeoutNs atomic.Int64 // time.Duration as nanoseconds
@@ -260,11 +291,13 @@ func (cn *Conn) CompareAndSwapUsed(old, new bool) bool {
260291

261292
if !old && new {
262293
// Acquiring: IDLE → IN_USE
263-
_, err := cn.stateMachine.TryTransition([]ConnState{StateIdle}, StateInUse)
294+
// Use predefined slice to avoid allocation
295+
_, err := cn.stateMachine.TryTransition(validFromCreatedOrIdle, StateInUse)
264296
return err == nil
265297
} else {
266298
// Releasing: IN_USE → IDLE
267-
_, err := cn.stateMachine.TryTransition([]ConnState{StateInUse}, StateIdle)
299+
// Use predefined slice to avoid allocation
300+
_, err := cn.stateMachine.TryTransition(validFromInUse, StateIdle)
268301
return err == nil
269302
}
270303
}
@@ -454,7 +487,8 @@ func (cn *Conn) getEffectiveReadTimeout(normalTimeout time.Duration) time.Durati
454487
return time.Duration(readTimeoutNs)
455488
}
456489

457-
nowNs := time.Now().UnixNano()
490+
// Use cached time to avoid expensive syscall (max 50ms staleness is acceptable for timeout checks)
491+
nowNs := getCachedTimeNs()
458492
// Check if deadline has passed
459493
if nowNs < deadlineNs {
460494
// Deadline is in the future, use relaxed timeout
@@ -487,7 +521,8 @@ func (cn *Conn) getEffectiveWriteTimeout(normalTimeout time.Duration) time.Durat
487521
return time.Duration(writeTimeoutNs)
488522
}
489523

490-
nowNs := time.Now().UnixNano()
524+
// Use cached time to avoid expensive syscall (max 50ms staleness is acceptable for timeout checks)
525+
nowNs := getCachedTimeNs()
491526
// Check if deadline has passed
492527
if nowNs < deadlineNs {
493528
// Deadline is in the future, use relaxed timeout
@@ -632,7 +667,8 @@ func (cn *Conn) MarkQueuedForHandoff() error {
632667
// The connection is typically in IN_USE state when OnPut is called (normal Put flow)
633668
// But in some edge cases or tests, it might be in IDLE or CREATED state
634669
// The pool will detect this state change and preserve it (not overwrite with IDLE)
635-
finalState, err := cn.stateMachine.TryTransition([]ConnState{StateInUse, StateIdle, StateCreated}, StateUnusable)
670+
// Use predefined slice to avoid allocation
671+
finalState, err := cn.stateMachine.TryTransition(validFromCreatedInUseOrIdle, StateUnusable)
636672
if err != nil {
637673
// Check if already in UNUSABLE state (race condition or retry)
638674
// ShouldHandoff should be false now, but check just in case
@@ -658,6 +694,42 @@ func (cn *Conn) GetStateMachine() *ConnStateMachine {
658694
return cn.stateMachine
659695
}
660696

697+
// TryAcquire attempts to acquire the connection for use.
698+
// This is an optimized inline method for the hot path (Get operation).
699+
//
700+
// It tries to transition from IDLE -> IN_USE or CREATED -> IN_USE.
701+
// Returns true if the connection was successfully acquired, false otherwise.
702+
//
703+
// Performance: This is faster than calling GetStateMachine() + TryTransitionFast()
704+
//
705+
// NOTE: We directly access cn.stateMachine.state here instead of using the state machine's
706+
// methods. This breaks encapsulation but is necessary for performance.
707+
// The IDLE->IN_USE and CREATED->IN_USE transitions don't need
708+
// waiter notification, and benchmarks show 1-3% improvement. If the state machine ever
709+
// needs to notify waiters on these transitions, update this to use TryTransitionFast().
710+
func (cn *Conn) TryAcquire() bool {
711+
// The || operator short-circuits, so only 1 CAS in the common case
712+
return cn.stateMachine.state.CompareAndSwap(uint32(StateIdle), uint32(StateInUse)) ||
713+
cn.stateMachine.state.CompareAndSwap(uint32(StateCreated), uint32(StateInUse))
714+
}
715+
716+
// Release releases the connection back to the pool.
717+
// This is an optimized inline method for the hot path (Put operation).
718+
//
719+
// It tries to transition from IN_USE -> IDLE.
720+
// Returns true if the connection was successfully released, false otherwise.
721+
//
722+
// Performance: This is faster than calling GetStateMachine() + TryTransitionFast().
723+
//
724+
// NOTE: We directly access cn.stateMachine.state here instead of using the state machine's
725+
// methods. This breaks encapsulation but is necessary for performance.
726+
// If the state machine ever needs to notify waiters
727+
// on this transition, update this to use TryTransitionFast().
728+
func (cn *Conn) Release() bool {
729+
// Inline the hot path - single CAS operation
730+
return cn.stateMachine.state.CompareAndSwap(uint32(StateInUse), uint32(StateIdle))
731+
}
732+
661733
// ClearHandoffState clears the handoff state after successful handoff.
662734
// Makes the connection usable again.
663735
func (cn *Conn) ClearHandoffState() {
@@ -800,8 +872,12 @@ func (cn *Conn) MaybeHasData() bool {
800872
return false
801873
}
802874

875+
// deadline computes the effective deadline time based on context and timeout.
876+
// It updates the usedAt timestamp to now.
877+
// Uses cached time to avoid expensive syscall (max 50ms staleness is acceptable for deadline calculation).
803878
func (cn *Conn) deadline(ctx context.Context, timeout time.Duration) time.Time {
804-
tm := time.Now()
879+
// Use cached time for deadline calculation (called 2x per command: read + write)
880+
tm := time.Unix(0, getCachedTimeNs())
805881
cn.SetUsedAt(tm)
806882

807883
if timeout > 0 {

0 commit comments

Comments
 (0)