1
0
mirror of https://github.com/redis/go-redis.git synced 2025-11-24 18:41:04 +03:00
Files
go-redis/maintnotifications/e2e/scenario_stress_test.go
Nedyalko Dyakov 042610b79d fix(conn): conn to have state machine (#3559)
* wip

* wip, used and unusable states

* polish state machine

* correct handling OnPut

* better errors for tests, hook should work now

* fix linter

* improve reauth state management. fix tests

* Update internal/pool/conn.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* Update internal/pool/conn.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* better timeouts

* empty endpoint handoff case

* fix handoff state when queued for handoff

* try to detect the deadlock

* try to detect the deadlock x2

* delete should be called

* improve tests

* fix mark on uninitialized connection

* Update internal/pool/conn_state_test.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* Update internal/pool/conn_state_test.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* Update internal/pool/pool.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* Update internal/pool/conn_state.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* Update internal/pool/conn.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* fix error from copilot

* address copilot comment

* 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>

* initConn sets IDLE state

- Handle unexpected conn state changes

* fix precision of time cache and usedAt

* allow e2e tests to run longer

* Fix broken initialization of idle connections

* optimize push notif

* 100ms -> 50ms

* use correct timer for last health check

* verify pass auth on conn creation

* fix assertion

* fix unsafe test

* fix benchmark test

* improve remove conn

* re doesn't support requirepass

* wait more in e2e test

* flaky test

* add missed method in interface

* fix test assertions

* silence logs and faster hooks manager

* address linter comment

* fix flaky test

* use read instad of control

* use pool size for semsize

* CAS instead of reading the state

* preallocate errors and states

* preallocate state slices

* fix flaky test

* fix fast semaphore that could have been starved

* try to fix the semaphore

* should properly notify the waiters

- this way a waiter that timesout at the same time
a releaser is releasing, won't throw token. the releaser
will fail to notify and will pick another waiter.

this hybrid approach should be faster than channels and maintains FIFO

* waiter may double-release (if closed/times out)

* priority of operations

* use simple approach of fifo waiters

* use simple channel based semaphores

* address linter and tests

* remove unused benchs

* change log message

* address pr comments

* address pr comments

* fix data race

---------

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
2025-11-11 17:38:29 +02:00

312 lines
9.2 KiB
Go

package e2e
import (
"context"
"fmt"
"os"
"sync"
"testing"
"time"
"github.com/redis/go-redis/v9"
"github.com/redis/go-redis/v9/logging"
"github.com/redis/go-redis/v9/maintnotifications"
)
// TestStressPushNotifications tests push notifications under extreme stress conditions
func TestStressPushNotifications(t *testing.T) {
if os.Getenv("E2E_SCENARIO_TESTS") != "true" {
t.Skip("[STRESS][SKIP] Scenario tests require E2E_SCENARIO_TESTS=true")
}
ctx, cancel := context.WithTimeout(context.Background(), 40*time.Minute)
defer cancel()
// Setup: Create fresh database and client factory for this test
bdbID, factory, cleanup := SetupTestDatabaseAndFactory(t, ctx, "standalone")
defer cleanup()
t.Logf("[STRESS] Created test database with bdb_id: %d", bdbID)
// Wait for database to be fully ready
time.Sleep(10 * time.Second)
var dump = true
var errorsDetected = false
var p = func(format string, args ...interface{}) {
printLog("STRESS", false, format, args...)
}
var e = func(format string, args ...interface{}) {
errorsDetected = true
printLog("STRESS", true, format, args...)
}
var ef = func(format string, args ...interface{}) {
printLog("STRESS", true, format, args...)
t.FailNow()
}
logCollector.ClearLogs()
defer func() {
logCollector.Clear()
}()
// Get endpoint config from factory (now connected to new database)
endpointConfig := factory.GetConfig()
// Create fault injector
faultInjector, err := CreateTestFaultInjector()
if err != nil {
ef("Failed to create fault injector: %v", err)
}
// Extreme stress configuration
minIdleConns := 50
poolSize := 150
maxConnections := 200
numClients := 4
var clients []redis.UniversalClient
var trackers []*TrackingNotificationsHook
var commandRunners []*CommandRunner
// Create multiple clients for extreme stress
for i := 0; i < numClients; i++ {
client, err := factory.Create(fmt.Sprintf("stress-client-%d", i), &CreateClientOptions{
Protocol: 3, // RESP3 required for push notifications
PoolSize: poolSize,
MinIdleConns: minIdleConns,
MaxActiveConns: maxConnections,
MaintNotificationsConfig: &maintnotifications.Config{
Mode: maintnotifications.ModeEnabled,
HandoffTimeout: 60 * time.Second, // Longer timeout for stress
RelaxedTimeout: 20 * time.Second, // Longer relaxed timeout
PostHandoffRelaxedDuration: 5 * time.Second, // Longer post-handoff duration
MaxWorkers: 50, // Maximum workers for stress
HandoffQueueSize: 1000, // Large queue for stress
EndpointType: maintnotifications.EndpointTypeExternalIP,
},
ClientName: fmt.Sprintf("stress-test-client-%d", i),
})
if err != nil {
ef("Failed to create stress client %d: %v", i, err)
}
clients = append(clients, client)
// Setup tracking for each client
tracker := NewTrackingNotificationsHook()
logger := maintnotifications.NewLoggingHook(int(logging.LogLevelWarn)) // Minimal logging for stress
setupNotificationHooks(client, tracker, logger)
trackers = append(trackers, tracker)
// Create command runner for each client
commandRunner, _ := NewCommandRunner(client)
commandRunners = append(commandRunners, commandRunner)
}
defer func() {
if dump {
p("Pool stats:")
factory.PrintPoolStats(t)
}
for _, runner := range commandRunners {
runner.Stop()
}
factory.DestroyAll()
}()
// Verify initial connectivity for all clients
for i, client := range clients {
err = client.Ping(ctx).Err()
if err != nil {
ef("Failed to ping Redis with stress client %d: %v", i, err)
}
}
p("All %d stress clients connected successfully", numClients)
// Start extreme traffic load on all clients
var trafficWg sync.WaitGroup
for i, runner := range commandRunners {
trafficWg.Add(1)
go func(clientID int, r *CommandRunner) {
defer trafficWg.Done()
p("Starting extreme traffic load on stress client %d", clientID)
r.FireCommandsUntilStop(ctx)
}(i, runner)
}
// Wait for traffic to stabilize
time.Sleep(10 * time.Second)
// Trigger multiple concurrent fault injection actions
var actionWg sync.WaitGroup
var actionResults []string
var actionMutex sync.Mutex
actions := []struct {
name string
action string
delay time.Duration
}{
{"failover-1", "failover", 0},
{"migrate-1", "migrate", 5 * time.Second},
{"failover-2", "failover", 10 * time.Second},
}
p("Starting %d concurrent fault injection actions under extreme stress...", len(actions))
for _, action := range actions {
actionWg.Add(1)
go func(actionName, actionType string, delay time.Duration) {
defer actionWg.Done()
if delay > 0 {
time.Sleep(delay)
}
p("Triggering %s action under extreme stress...", actionName)
var resp *ActionResponse
var err error
switch actionType {
case "failover":
resp, err = faultInjector.TriggerAction(ctx, ActionRequest{
Type: "failover",
Parameters: map[string]interface{}{
"bdb_id": endpointConfig.BdbID,
},
})
case "migrate":
resp, err = faultInjector.TriggerAction(ctx, ActionRequest{
Type: "migrate",
Parameters: map[string]interface{}{
"bdb_id": endpointConfig.BdbID,
},
})
}
if err != nil {
e("Failed to trigger %s action: %v", actionName, err)
return
}
// Wait for action to complete
status, err := faultInjector.WaitForAction(ctx, resp.ActionID,
WithMaxWaitTime(360*time.Second), // Longer wait time for stress
WithPollInterval(2*time.Second),
)
if err != nil {
e("[FI] %s action failed: %v", actionName, err)
return
}
actionMutex.Lock()
actionResults = append(actionResults, fmt.Sprintf("%s: %s %s", actionName, status.Status, actionOutputIfFailed(status)))
actionMutex.Unlock()
p("[FI] %s action completed: %s %s", actionName, status.Status, actionOutputIfFailed(status))
}(action.name, action.action, action.delay)
}
// Wait for all actions to complete
actionWg.Wait()
// Continue stress for a bit longer
p("All fault injection actions completed, continuing stress for 2 more minutes...")
time.Sleep(2 * time.Minute)
// Stop all command runners
for _, runner := range commandRunners {
runner.Stop()
}
trafficWg.Wait()
// Analyze stress test results
allLogsAnalysis := logCollector.GetAnalysis()
totalOperations := int64(0)
totalErrors := int64(0)
totalTimeoutErrors := int64(0)
for i, runner := range commandRunners {
stats := runner.GetStats()
p("Stress client %d stats: Operations: %d, Errors: %d, Timeout Errors: %d",
i, stats.Operations, stats.Errors, stats.TimeoutErrors)
totalOperations += stats.Operations
totalErrors += stats.Errors
totalTimeoutErrors += stats.TimeoutErrors
}
p("STRESS TEST RESULTS:")
p("Total operations across all clients: %d", totalOperations)
p("Total errors: %d (%.2f%%)", totalErrors, float64(totalErrors)/float64(totalOperations)*100)
p("Total timeout errors: %d (%.2f%%)", totalTimeoutErrors, float64(totalTimeoutErrors)/float64(totalOperations)*100)
p("Total connections used: %d", allLogsAnalysis.ConnectionCount)
// Print action results
actionMutex.Lock()
p("Fault injection action results:")
for _, result := range actionResults {
p(" %s", result)
}
actionMutex.Unlock()
// Validate stress test results
if totalOperations < 1000 {
e("Expected at least 1000 operations under stress, got %d", totalOperations)
}
// Allow higher error rates under extreme stress (up to 20%)
errorRate := float64(totalErrors) / float64(totalOperations) * 100
if errorRate > 20.0 {
e("Error rate too high under stress: %.2f%% (max allowed: 20%%)", errorRate)
}
// Validate connection limits weren't exceeded
expectedMaxConnections := int64(numClients * maxConnections)
if allLogsAnalysis.ConnectionCount > expectedMaxConnections {
e("Connection count exceeded limit: %d > %d", allLogsAnalysis.ConnectionCount, expectedMaxConnections)
}
// Validate notifications were processed
totalTrackerNotifications := int64(0)
totalProcessingErrors := int64(0)
for _, tracker := range trackers {
analysis := tracker.GetAnalysis()
totalTrackerNotifications += analysis.TotalNotifications
totalProcessingErrors += analysis.NotificationProcessingErrors
}
if totalProcessingErrors > totalTrackerNotifications/10 { // Allow up to 10% processing errors under stress
e("Too many notification processing errors under stress: %d/%d", totalProcessingErrors, totalTrackerNotifications)
}
if errorsDetected {
ef("Errors detected under stress")
logCollector.DumpLogs()
for i, tracker := range trackers {
p("=== Stress Client %d Analysis ===", i)
tracker.GetAnalysis().Print(t)
}
logCollector.Clear()
for _, tracker := range trackers {
tracker.Clear()
}
}
dump = false
p("[SUCCESS] Stress test completed successfully!")
p("Processed %d operations across %d clients with %d connections",
totalOperations, numClients, allLogsAnalysis.ConnectionCount)
p("Error rate: %.2f%%, Notification processing errors: %d/%d",
errorRate, totalProcessingErrors, totalTrackerNotifications)
// Print final analysis
allLogsAnalysis.Print(t)
for i, tracker := range trackers {
p("=== Stress Client %d Analysis ===", i)
tracker.GetAnalysis().Print(t)
}
}