1
0
mirror of https://github.com/redis/go-redis.git synced 2025-12-02 06:22:31 +03:00

feat: add optional logger wherever possible

This commit introduces an optional logger parameter to various structs.
This enhancement allows users to provide custom logging implementations.
This commit is contained in:
ccoVeille
2025-10-24 14:59:40 +02:00
parent 5b0b228a37
commit c98107019e
15 changed files with 340 additions and 172 deletions

View File

@@ -77,3 +77,66 @@ func (l LogLevelT) InfoOrAbove() bool {
func (l LogLevelT) DebugOrAbove() bool { func (l LogLevelT) DebugOrAbove() bool {
return l >= LogLevelDebug return l >= LogLevelDebug
} }
// LoggerWithLevel is a logger interface with leveled logging methods.
//
// This interface can be implemented by custom loggers to provide leveled logging.
type LoggerWithLevel interface {
// Infof logs an info level message
Infof(ctx context.Context, format string, v ...interface{})
// Warnf logs a warning level message
Warnf(ctx context.Context, format string, v ...interface{})
// Debugf logs a debug level message
Debugf(ctx context.Context, format string, v ...interface{})
// Errorf logs an error level message
Errorf(ctx context.Context, format string, v ...interface{})
// Enabled reports whether the given log level is enabled in the logger
Enabled(ctx context.Context, level LogLevelT) bool
}
// legacyLoggerAdapter is a logger that implements LoggerWithLevel interface
// using the global [Logger] and [LogLevel] variables.
type legacyLoggerAdapter struct{}
func (l *legacyLoggerAdapter) Infof(ctx context.Context, format string, v ...interface{}) {
if LogLevel.InfoOrAbove() {
Logger.Printf(ctx, format, v...)
}
}
func (l *legacyLoggerAdapter) Warnf(ctx context.Context, format string, v ...interface{}) {
if LogLevel.WarnOrAbove() {
Logger.Printf(ctx, format, v...)
}
}
func (l *legacyLoggerAdapter) Debugf(ctx context.Context, format string, v ...interface{}) {
if LogLevel.DebugOrAbove() {
Logger.Printf(ctx, format, v...)
}
}
func (l legacyLoggerAdapter) Errorf(ctx context.Context, format string, v ...interface{}) {
Logger.Printf(ctx, format, v...)
}
func (l legacyLoggerAdapter) Enabled(_ context.Context, level LogLevelT) bool {
switch level {
case LogLevelWarn:
return LogLevel.WarnOrAbove()
case LogLevelInfo:
return LogLevel.InfoOrAbove()
case LogLevelDebug:
return LogLevel.DebugOrAbove()
case LogLevelError:
fallthrough
default:
return true
}
}
var LegacyLoggerWithLevel LoggerWithLevel = &legacyLoggerAdapter{}

View File

@@ -119,6 +119,9 @@ type Options struct {
// DialerRetryTimeout is the backoff duration between retry attempts. // DialerRetryTimeout is the backoff duration between retry attempts.
// Default: 100ms // Default: 100ms
DialerRetryTimeout time.Duration DialerRetryTimeout time.Duration
// Optional logger for connection pool operations.
Logger internal.LoggerWithLevel
} }
type lastDialErrorWrap struct { type lastDialErrorWrap struct {
@@ -257,7 +260,7 @@ func (p *ConnPool) checkMinIdleConns() {
p.idleConnsLen.Add(-1) p.idleConnsLen.Add(-1)
p.freeTurn() p.freeTurn()
internal.Logger.Printf(context.Background(), "addIdleConn panic: %+v", err) p.logger().Errorf(context.Background(), "addIdleConn panic: %+v", err)
} }
}() }()
@@ -419,7 +422,7 @@ func (p *ConnPool) dialConn(ctx context.Context, pooled bool) (*Conn, error) {
return cn, nil return cn, nil
} }
internal.Logger.Printf(ctx, "redis: connection pool: failed to dial after %d attempts: %v", attempt, lastErr) p.logger().Errorf(ctx, "redis: connection pool: failed to dial after %d attempts: %v", attempt, lastErr)
// All retries failed - handle error tracking // All retries failed - handle error tracking
p.setLastDialError(lastErr) p.setLastDialError(lastErr)
if atomic.AddUint32(&p.dialErrorsNum, 1) == uint32(p.cfg.PoolSize) { if atomic.AddUint32(&p.dialErrorsNum, 1) == uint32(p.cfg.PoolSize) {
@@ -513,10 +516,10 @@ func (p *ConnPool) getConn(ctx context.Context) (*Conn, error) {
acceptConn, err := hookManager.ProcessOnGet(ctx, cn, false) acceptConn, err := hookManager.ProcessOnGet(ctx, cn, false)
if err != nil || !acceptConn { if err != nil || !acceptConn {
if err != nil { if err != nil {
internal.Logger.Printf(ctx, "redis: connection pool: failed to process idle connection by hook: %v", err) p.logger().Errorf(ctx, "redis: connection pool: failed to process idle connection by hook: %v", err)
_ = p.CloseConn(cn) _ = p.CloseConn(cn)
} else { } else {
internal.Logger.Printf(ctx, "redis: connection pool: conn[%d] rejected by hook, returning to pool", cn.GetID()) p.logger().Errorf(ctx, "redis: connection pool: conn[%d] rejected by hook, returning to pool", cn.GetID())
// Return connection to pool without freeing the turn that this Get() call holds. // Return connection to pool without freeing the turn that this Get() call holds.
// We use putConnWithoutTurn() to run all the Put hooks and logic without freeing a turn. // We use putConnWithoutTurn() to run all the Put hooks and logic without freeing a turn.
p.putConnWithoutTurn(ctx, cn) p.putConnWithoutTurn(ctx, cn)
@@ -544,7 +547,7 @@ func (p *ConnPool) getConn(ctx context.Context) (*Conn, error) {
// this should not happen with a new connection, but we handle it gracefully // this should not happen with a new connection, but we handle it gracefully
if err != nil || !acceptConn { if err != nil || !acceptConn {
// Failed to process connection, discard it // Failed to process connection, discard it
internal.Logger.Printf(ctx, "redis: connection pool: failed to process new connection conn[%d] by hook: accept=%v, err=%v", newcn.GetID(), acceptConn, err) p.logger().Errorf(ctx, "redis: connection pool: failed to process new connection conn[%d] by hook: accept=%v, err=%v", newcn.GetID(), acceptConn, err)
_ = p.CloseConn(newcn) _ = p.CloseConn(newcn)
return nil, err return nil, err
} }
@@ -587,7 +590,7 @@ func (p *ConnPool) queuedNewConn(ctx context.Context) (*Conn, error) {
if !freeTurnCalled { if !freeTurnCalled {
p.freeTurn() p.freeTurn()
} }
internal.Logger.Printf(context.Background(), "queuedNewConn panic: %+v", err) p.logger().Errorf(ctx, "queuedNewConn panic: %+v", err)
} }
}() }()
@@ -731,7 +734,7 @@ func (p *ConnPool) popIdle() (*Conn, error) {
// If we exhausted all attempts without finding a usable connection, return nil // If we exhausted all attempts without finding a usable connection, return nil
if attempts > 1 && attempts >= maxAttempts && int32(attempts) >= p.poolSize.Load() { if attempts > 1 && attempts >= maxAttempts && int32(attempts) >= p.poolSize.Load() {
internal.Logger.Printf(context.Background(), "redis: connection pool: failed to get a usable connection after %d attempts", attempts) p.logger().Errorf(context.Background(), "redis: connection pool: failed to get a usable connection after %d attempts", attempts)
return nil, nil return nil, nil
} }
@@ -760,7 +763,7 @@ func (p *ConnPool) putConn(ctx context.Context, cn *Conn, freeTurn bool) {
// Peek at the reply type to check if it's a push notification // Peek at the reply type to check if it's a push notification
if replyType, err := cn.PeekReplyTypeSafe(); err != nil || replyType != proto.RespPush { if replyType, err := cn.PeekReplyTypeSafe(); err != nil || replyType != proto.RespPush {
// Not a push notification or error peeking, remove connection // Not a push notification or error peeking, remove connection
internal.Logger.Printf(ctx, "Conn has unread data (not push notification), removing it") p.logger().Errorf(ctx, "Conn has unread data (not push notification), removing it")
p.removeConnInternal(ctx, cn, err, freeTurn) p.removeConnInternal(ctx, cn, err, freeTurn)
return return
} }
@@ -773,7 +776,7 @@ func (p *ConnPool) putConn(ctx context.Context, cn *Conn, freeTurn bool) {
if hookManager != nil { if hookManager != nil {
shouldPool, shouldRemove, err = hookManager.ProcessOnPut(ctx, cn) shouldPool, shouldRemove, err = hookManager.ProcessOnPut(ctx, cn)
if err != nil { if err != nil {
internal.Logger.Printf(ctx, "Connection hook error: %v", err) p.logger().Errorf(ctx, "Connection hook error: %v", err)
p.removeConnInternal(ctx, cn, err, freeTurn) p.removeConnInternal(ctx, cn, err, freeTurn)
return return
} }
@@ -806,12 +809,12 @@ func (p *ConnPool) putConn(ctx context.Context, cn *Conn, freeTurn bool) {
case StateUnusable: case StateUnusable:
// expected state, don't log it // expected state, don't log it
case StateClosed: case StateClosed:
internal.Logger.Printf(ctx, "Unexpected conn[%d] state changed by hook to %v, closing it", cn.GetID(), currentState) p.logger().Errorf(ctx, "Unexpected conn[%d] state changed by hook to %v, closing it", cn.GetID(), currentState)
shouldCloseConn = true shouldCloseConn = true
p.removeConnWithLock(cn) p.removeConnWithLock(cn)
default: default:
// Pool as-is // Pool as-is
internal.Logger.Printf(ctx, "Unexpected conn[%d] state changed by hook to %v, pooling as-is", cn.GetID(), currentState) p.logger().Warnf(ctx, "Unexpected conn[%d] state changed by hook to %v, pooling as-is", cn.GetID(), currentState)
} }
} }
@@ -1028,7 +1031,7 @@ func (p *ConnPool) isHealthyConn(cn *Conn, nowNs int64) bool {
if replyType, err := cn.rd.PeekReplyType(); err == nil && replyType == proto.RespPush { if replyType, err := cn.rd.PeekReplyType(); err == nil && replyType == proto.RespPush {
// For RESP3 connections with push notifications, we allow some buffered data // For RESP3 connections with push notifications, we allow some buffered data
// The client will process these notifications before using the connection // The client will process these notifications before using the connection
internal.Logger.Printf( p.logger().Infof(
context.Background(), context.Background(),
"push: conn[%d] has buffered data, likely push notifications - will be processed by client", "push: conn[%d] has buffered data, likely push notifications - will be processed by client",
cn.GetID(), cn.GetID(),
@@ -1051,3 +1054,11 @@ func (p *ConnPool) isHealthyConn(cn *Conn, nowNs int64) bool {
cn.SetUsedAtNs(nowNs) cn.SetUsedAtNs(nowNs)
return true return true
} }
func (p *ConnPool) logger() internal.LoggerWithLevel {
if p.cfg.Logger != nil {
return p.cfg.Logger
}
return internal.LegacyLoggerWithLevel
}

View File

@@ -89,3 +89,4 @@ func (l *filterLogger) Printf(ctx context.Context, format string, v ...interface
return return
} }
} }

View File

@@ -102,9 +102,7 @@ func (cb *CircuitBreaker) Execute(fn func() error) error {
if cb.state.CompareAndSwap(int32(CircuitBreakerOpen), int32(CircuitBreakerHalfOpen)) { if cb.state.CompareAndSwap(int32(CircuitBreakerOpen), int32(CircuitBreakerHalfOpen)) {
cb.requests.Store(0) cb.requests.Store(0)
cb.successes.Store(0) cb.successes.Store(0)
if internal.LogLevel.InfoOrAbove() { cb.logger().Infof(context.Background(), logs.CircuitBreakerTransitioningToHalfOpen(cb.endpoint))
internal.Logger.Printf(context.Background(), logs.CircuitBreakerTransitioningToHalfOpen(cb.endpoint))
}
// Fall through to half-open logic // Fall through to half-open logic
} else { } else {
return ErrCircuitBreakerOpen return ErrCircuitBreakerOpen
@@ -144,17 +142,13 @@ func (cb *CircuitBreaker) recordFailure() {
case CircuitBreakerClosed: case CircuitBreakerClosed:
if failures >= int64(cb.failureThreshold) { if failures >= int64(cb.failureThreshold) {
if cb.state.CompareAndSwap(int32(CircuitBreakerClosed), int32(CircuitBreakerOpen)) { if cb.state.CompareAndSwap(int32(CircuitBreakerClosed), int32(CircuitBreakerOpen)) {
if internal.LogLevel.WarnOrAbove() { cb.logger().Warnf(context.Background(), logs.CircuitBreakerOpened(cb.endpoint, failures))
internal.Logger.Printf(context.Background(), logs.CircuitBreakerOpened(cb.endpoint, failures))
}
} }
} }
case CircuitBreakerHalfOpen: case CircuitBreakerHalfOpen:
// Any failure in half-open state immediately opens the circuit // Any failure in half-open state immediately opens the circuit
if cb.state.CompareAndSwap(int32(CircuitBreakerHalfOpen), int32(CircuitBreakerOpen)) { if cb.state.CompareAndSwap(int32(CircuitBreakerHalfOpen), int32(CircuitBreakerOpen)) {
if internal.LogLevel.WarnOrAbove() { cb.logger().Warnf(context.Background(), logs.CircuitBreakerReopened(cb.endpoint))
internal.Logger.Printf(context.Background(), logs.CircuitBreakerReopened(cb.endpoint))
}
} }
} }
} }
@@ -176,9 +170,7 @@ func (cb *CircuitBreaker) recordSuccess() {
if successes >= int64(cb.maxRequests) { if successes >= int64(cb.maxRequests) {
if cb.state.CompareAndSwap(int32(CircuitBreakerHalfOpen), int32(CircuitBreakerClosed)) { if cb.state.CompareAndSwap(int32(CircuitBreakerHalfOpen), int32(CircuitBreakerClosed)) {
cb.failures.Store(0) cb.failures.Store(0)
if internal.LogLevel.InfoOrAbove() { cb.logger().Infof(context.Background(), logs.CircuitBreakerClosed(cb.endpoint, successes))
internal.Logger.Printf(context.Background(), logs.CircuitBreakerClosed(cb.endpoint, successes))
}
} }
} }
} }
@@ -202,6 +194,13 @@ func (cb *CircuitBreaker) GetStats() CircuitBreakerStats {
} }
} }
func (cb *CircuitBreaker) logger() internal.LoggerWithLevel {
if cb.config != nil && cb.config.Logger != nil {
return cb.config.Logger
}
return internal.LegacyLoggerWithLevel
}
// CircuitBreakerStats provides statistics about a circuit breaker // CircuitBreakerStats provides statistics about a circuit breaker
type CircuitBreakerStats struct { type CircuitBreakerStats struct {
Endpoint string Endpoint string
@@ -325,8 +324,8 @@ func (cbm *CircuitBreakerManager) cleanup() {
} }
// Log cleanup results // Log cleanup results
if len(toDelete) > 0 && internal.LogLevel.InfoOrAbove() { if len(toDelete) > 0 {
internal.Logger.Printf(context.Background(), logs.CircuitBreakerCleanup(len(toDelete), count)) cbm.logger().Infof(context.Background(), logs.CircuitBreakerCleanup(len(toDelete), count))
} }
cbm.lastCleanup.Store(now.Unix()) cbm.lastCleanup.Store(now.Unix())
@@ -351,3 +350,10 @@ func (cbm *CircuitBreakerManager) Reset() {
return true return true
}) })
} }
func (cbm *CircuitBreakerManager) logger() internal.LoggerWithLevel {
if cbm.config != nil && cbm.config.Logger != nil {
return cbm.config.Logger
}
return internal.LegacyLoggerWithLevel
}

View File

@@ -128,6 +128,9 @@ type Config struct {
// After this many retries, the connection will be removed from the pool. // After this many retries, the connection will be removed from the pool.
// Default: 3 // Default: 3
MaxHandoffRetries int MaxHandoffRetries int
// Logger is an optional custom logger for maintenance notifications.
Logger internal.LoggerWithLevel
} }
func (c *Config) IsEnabled() bool { func (c *Config) IsEnabled() bool {
@@ -312,10 +315,9 @@ func (c *Config) ApplyDefaultsWithPoolConfig(poolSize int, maxActiveConns int) *
result.CircuitBreakerMaxRequests = c.CircuitBreakerMaxRequests result.CircuitBreakerMaxRequests = c.CircuitBreakerMaxRequests
} }
if internal.LogLevel.DebugOrAbove() { c.logger().Debugf(context.Background(), logs.DebugLoggingEnabled())
internal.Logger.Printf(context.Background(), logs.DebugLoggingEnabled()) c.logger().Debugf(context.Background(), logs.ConfigDebug(result))
internal.Logger.Printf(context.Background(), logs.ConfigDebug(result))
}
return result return result
} }
@@ -341,6 +343,8 @@ func (c *Config) Clone() *Config {
// Configuration fields // Configuration fields
MaxHandoffRetries: c.MaxHandoffRetries, MaxHandoffRetries: c.MaxHandoffRetries,
Logger: c.Logger,
} }
} }
@@ -365,6 +369,13 @@ func (c *Config) applyWorkerDefaults(poolSize int) {
} }
} }
func (c *Config) logger() internal.LoggerWithLevel {
if c.Logger != nil {
return c.Logger
}
return internal.LegacyLoggerWithLevel
}
// DetectEndpointType automatically detects the appropriate endpoint type // DetectEndpointType automatically detects the appropriate endpoint type
// based on the connection address and TLS configuration. // based on the connection address and TLS configuration.
// //

View File

@@ -121,7 +121,7 @@ func (hwm *handoffWorkerManager) onDemandWorker() {
defer func() { defer func() {
// Handle panics to ensure proper cleanup // Handle panics to ensure proper cleanup
if r := recover(); r != nil { if r := recover(); r != nil {
internal.Logger.Printf(context.Background(), logs.WorkerPanicRecovered(r)) hwm.logger().Errorf(context.Background(), logs.WorkerPanicRecovered(r))
} }
// Decrement active worker count when exiting // Decrement active worker count when exiting
@@ -145,23 +145,17 @@ func (hwm *handoffWorkerManager) onDemandWorker() {
select { select {
case <-hwm.shutdown: case <-hwm.shutdown:
if internal.LogLevel.InfoOrAbove() { hwm.logger().Infof(context.Background(), logs.WorkerExitingDueToShutdown())
internal.Logger.Printf(context.Background(), logs.WorkerExitingDueToShutdown())
}
return return
case <-timer.C: case <-timer.C:
// Worker has been idle for too long, exit to save resources // Worker has been idle for too long, exit to save resources
if internal.LogLevel.InfoOrAbove() { hwm.logger().Infof(context.Background(), logs.WorkerExitingDueToInactivityTimeout(hwm.workerTimeout))
internal.Logger.Printf(context.Background(), logs.WorkerExitingDueToInactivityTimeout(hwm.workerTimeout))
}
return return
case request := <-hwm.handoffQueue: case request := <-hwm.handoffQueue:
// Check for shutdown before processing // Check for shutdown before processing
select { select {
case <-hwm.shutdown: case <-hwm.shutdown:
if internal.LogLevel.InfoOrAbove() { hwm.logger().Infof(context.Background(), logs.WorkerExitingDueToShutdownWhileProcessing())
internal.Logger.Printf(context.Background(), logs.WorkerExitingDueToShutdownWhileProcessing())
}
// Clean up the request before exiting // Clean up the request before exiting
hwm.pending.Delete(request.ConnID) hwm.pending.Delete(request.ConnID)
return return
@@ -175,9 +169,7 @@ func (hwm *handoffWorkerManager) onDemandWorker() {
// processHandoffRequest processes a single handoff request // processHandoffRequest processes a single handoff request
func (hwm *handoffWorkerManager) processHandoffRequest(request HandoffRequest) { func (hwm *handoffWorkerManager) processHandoffRequest(request HandoffRequest) {
if internal.LogLevel.InfoOrAbove() { hwm.logger().Infof(context.Background(), logs.HandoffStarted(request.Conn.GetID(), request.Endpoint))
internal.Logger.Printf(context.Background(), logs.HandoffStarted(request.Conn.GetID(), request.Endpoint))
}
// Create a context with handoff timeout from config // Create a context with handoff timeout from config
handoffTimeout := 15 * time.Second // Default timeout handoffTimeout := 15 * time.Second // Default timeout
@@ -217,21 +209,22 @@ func (hwm *handoffWorkerManager) processHandoffRequest(request HandoffRequest) {
afterTime = minRetryBackoff afterTime = minRetryBackoff
} }
if internal.LogLevel.InfoOrAbove() { // the HandoffRetries() requires locking resource via [atomic.Uint32.Load],
// so we check the log level first before calling it
if hwm.logger().Enabled(context.Background(), internal.LogLevelInfo) {
// Get current retry count for better logging // Get current retry count for better logging
currentRetries := request.Conn.HandoffRetries() currentRetries := request.Conn.HandoffRetries()
maxRetries := 3 // Default fallback maxRetries := 3 // Default fallback
if hwm.config != nil { if hwm.config != nil {
maxRetries = hwm.config.MaxHandoffRetries maxRetries = hwm.config.MaxHandoffRetries
} }
internal.Logger.Printf(context.Background(), logs.HandoffFailed(request.ConnID, request.Endpoint, currentRetries, maxRetries, err)) hwm.logger().Infof(context.Background(), logs.HandoffFailed(request.ConnID, request.Endpoint, currentRetries, maxRetries, err))
} }
// Schedule retry - keep connection in pending map until retry is queued // Schedule retry - keep connection in pending map until retry is queued
time.AfterFunc(afterTime, func() { time.AfterFunc(afterTime, func() {
if err := hwm.queueHandoff(request.Conn); err != nil { if err := hwm.queueHandoff(request.Conn); err != nil {
if internal.LogLevel.WarnOrAbove() { hwm.logger().Warnf(context.Background(), logs.CannotQueueHandoffForRetry(err))
internal.Logger.Printf(context.Background(), logs.CannotQueueHandoffForRetry(err))
}
// Failed to queue retry - remove from pending and close connection // Failed to queue retry - remove from pending and close connection
hwm.pending.Delete(request.Conn.GetID()) hwm.pending.Delete(request.Conn.GetID())
hwm.closeConnFromRequest(context.Background(), request, err) hwm.closeConnFromRequest(context.Background(), request, err)
@@ -268,9 +261,7 @@ func (hwm *handoffWorkerManager) queueHandoff(conn *pool.Conn) error {
// on retries the connection will not be marked for handoff, but it will have retries > 0 // on retries the connection will not be marked for handoff, but it will have retries > 0
// if shouldHandoff is false and retries is 0, then we are not retrying and not do a handoff // if shouldHandoff is false and retries is 0, then we are not retrying and not do a handoff
if !shouldHandoff && conn.HandoffRetries() == 0 { if !shouldHandoff && conn.HandoffRetries() == 0 {
if internal.LogLevel.InfoOrAbove() { hwm.logger().Infof(context.Background(), logs.ConnectionNotMarkedForHandoff(conn.GetID()))
internal.Logger.Printf(context.Background(), logs.ConnectionNotMarkedForHandoff(conn.GetID()))
}
return errors.New(logs.ConnectionNotMarkedForHandoffError(conn.GetID())) return errors.New(logs.ConnectionNotMarkedForHandoffError(conn.GetID()))
} }
@@ -311,9 +302,7 @@ func (hwm *handoffWorkerManager) queueHandoff(conn *pool.Conn) error {
// Queue is full - log and attempt scaling // Queue is full - log and attempt scaling
queueLen := len(hwm.handoffQueue) queueLen := len(hwm.handoffQueue)
queueCap := cap(hwm.handoffQueue) queueCap := cap(hwm.handoffQueue)
if internal.LogLevel.WarnOrAbove() { hwm.logger().Warnf(context.Background(), logs.HandoffQueueFull(queueLen, queueCap))
internal.Logger.Printf(context.Background(), logs.HandoffQueueFull(queueLen, queueCap))
}
} }
} }
} }
@@ -366,7 +355,7 @@ func (hwm *handoffWorkerManager) performConnectionHandoff(ctx context.Context, c
// Check if circuit breaker is open before attempting handoff // Check if circuit breaker is open before attempting handoff
if circuitBreaker.IsOpen() { if circuitBreaker.IsOpen() {
internal.Logger.Printf(ctx, logs.CircuitBreakerOpen(connID, newEndpoint)) hwm.logger().Infof(ctx, logs.CircuitBreakerOpen(connID, newEndpoint))
return false, ErrCircuitBreakerOpen // Don't retry when circuit breaker is open return false, ErrCircuitBreakerOpen // Don't retry when circuit breaker is open
} }
@@ -395,16 +384,14 @@ func (hwm *handoffWorkerManager) performHandoffInternal(
connID uint64, connID uint64,
) (shouldRetry bool, err error) { ) (shouldRetry bool, err error) {
retries := conn.IncrementAndGetHandoffRetries(1) retries := conn.IncrementAndGetHandoffRetries(1)
internal.Logger.Printf(ctx, logs.HandoffRetryAttempt(connID, retries, newEndpoint, conn.RemoteAddr().String())) hwm.logger().Infof(ctx, logs.HandoffRetryAttempt(connID, retries, newEndpoint, conn.RemoteAddr().String()))
maxRetries := 3 // Default fallback maxRetries := 3 // Default fallback
if hwm.config != nil { if hwm.config != nil {
maxRetries = hwm.config.MaxHandoffRetries maxRetries = hwm.config.MaxHandoffRetries
} }
if retries > maxRetries { if retries > maxRetries {
if internal.LogLevel.WarnOrAbove() { hwm.logger().Warnf(ctx, logs.ReachedMaxHandoffRetries(connID, newEndpoint, maxRetries))
internal.Logger.Printf(ctx, logs.ReachedMaxHandoffRetries(connID, newEndpoint, maxRetries))
}
// won't retry on ErrMaxHandoffRetriesReached // won't retry on ErrMaxHandoffRetriesReached
return false, ErrMaxHandoffRetriesReached return false, ErrMaxHandoffRetriesReached
} }
@@ -415,7 +402,7 @@ func (hwm *handoffWorkerManager) performHandoffInternal(
// Create new connection to the new endpoint // Create new connection to the new endpoint
newNetConn, err := endpointDialer(ctx) newNetConn, err := endpointDialer(ctx)
if err != nil { if err != nil {
internal.Logger.Printf(ctx, logs.FailedToDialNewEndpoint(connID, newEndpoint, err)) hwm.logger().Errorf(ctx, logs.FailedToDialNewEndpoint(connID, newEndpoint, err))
// will retry // will retry
// Maybe a network error - retry after a delay // Maybe a network error - retry after a delay
return true, err return true, err
@@ -434,9 +421,7 @@ func (hwm *handoffWorkerManager) performHandoffInternal(
deadline := time.Now().Add(hwm.config.PostHandoffRelaxedDuration) deadline := time.Now().Add(hwm.config.PostHandoffRelaxedDuration)
conn.SetRelaxedTimeoutWithDeadline(relaxedTimeout, relaxedTimeout, deadline) conn.SetRelaxedTimeoutWithDeadline(relaxedTimeout, relaxedTimeout, deadline)
if internal.LogLevel.InfoOrAbove() { hwm.logger().Infof(context.Background(), logs.ApplyingRelaxedTimeoutDueToPostHandoff(connID, relaxedTimeout, deadline.Format("15:04:05.000")))
internal.Logger.Printf(context.Background(), logs.ApplyingRelaxedTimeoutDueToPostHandoff(connID, relaxedTimeout, deadline.Format("15:04:05.000")))
}
} }
// Replace the connection and execute initialization // Replace the connection and execute initialization
@@ -459,7 +444,7 @@ func (hwm *handoffWorkerManager) performHandoffInternal(
// Note: Theoretically there may be a short window where the connection is in the pool // Note: Theoretically there may be a short window where the connection is in the pool
// and IDLE (initConn completed) but still has handoff state set. // and IDLE (initConn completed) but still has handoff state set.
conn.ClearHandoffState() conn.ClearHandoffState()
internal.Logger.Printf(ctx, logs.HandoffSucceeded(connID, newEndpoint)) hwm.logger().Infof(ctx, logs.HandoffSucceeded(connID, newEndpoint))
// successfully completed the handoff, no retry needed and no error // successfully completed the handoff, no retry needed and no error
return false, nil return false, nil
@@ -497,16 +482,19 @@ func (hwm *handoffWorkerManager) closeConnFromRequest(ctx context.Context, reque
// Remove() is meant to be called after Get() and frees a turn. // Remove() is meant to be called after Get() and frees a turn.
// RemoveWithoutTurn() removes and closes the connection without affecting the queue. // RemoveWithoutTurn() removes and closes the connection without affecting the queue.
pooler.RemoveWithoutTurn(ctx, conn, err) pooler.RemoveWithoutTurn(ctx, conn, err)
if internal.LogLevel.WarnOrAbove() { hwm.logger().Warnf(ctx, logs.RemovingConnectionFromPool(conn.GetID(), err))
internal.Logger.Printf(ctx, logs.RemovingConnectionFromPool(conn.GetID(), err))
}
} else { } else {
err := conn.Close() // Close the connection if no pool provided err := conn.Close() // Close the connection if no pool provided
if err != nil { if err != nil {
internal.Logger.Printf(ctx, "redis: failed to close connection: %v", err) hwm.logger().Errorf(ctx, "redis: failed to close connection: %v", err)
} }
if internal.LogLevel.WarnOrAbove() { hwm.logger().Warnf(ctx, logs.NoPoolProvidedCannotRemove(conn.GetID(), err))
internal.Logger.Printf(ctx, logs.NoPoolProvidedCannotRemove(conn.GetID(), err))
} }
} }
func (hwm *handoffWorkerManager) logger() internal.LoggerWithLevel {
if hwm.config != nil && hwm.config.Logger != nil {
return hwm.config.Logger
}
return internal.LegacyLoggerWithLevel
} }

View File

@@ -150,14 +150,10 @@ func (hm *Manager) TrackMovingOperationWithConnID(ctx context.Context, newEndpoi
// Use LoadOrStore for atomic check-and-set operation // Use LoadOrStore for atomic check-and-set operation
if _, loaded := hm.activeMovingOps.LoadOrStore(key, movingOp); loaded { if _, loaded := hm.activeMovingOps.LoadOrStore(key, movingOp); loaded {
// Duplicate MOVING notification, ignore // Duplicate MOVING notification, ignore
if internal.LogLevel.DebugOrAbove() { // Debug level hm.logger().Debugf(context.Background(), logs.DuplicateMovingOperation(connID, newEndpoint, seqID))
internal.Logger.Printf(context.Background(), logs.DuplicateMovingOperation(connID, newEndpoint, seqID))
}
return nil return nil
} }
if internal.LogLevel.DebugOrAbove() { // Debug level hm.logger().Debugf(context.Background(), logs.TrackingMovingOperation(connID, newEndpoint, seqID))
internal.Logger.Printf(context.Background(), logs.TrackingMovingOperation(connID, newEndpoint, seqID))
}
// Increment active operation count atomically // Increment active operation count atomically
hm.activeOperationCount.Add(1) hm.activeOperationCount.Add(1)
@@ -175,15 +171,11 @@ func (hm *Manager) UntrackOperationWithConnID(seqID int64, connID uint64) {
// Remove from active operations atomically // Remove from active operations atomically
if _, loaded := hm.activeMovingOps.LoadAndDelete(key); loaded { if _, loaded := hm.activeMovingOps.LoadAndDelete(key); loaded {
if internal.LogLevel.DebugOrAbove() { // Debug level hm.logger().Debugf(context.Background(), logs.UntrackingMovingOperation(connID, seqID))
internal.Logger.Printf(context.Background(), logs.UntrackingMovingOperation(connID, seqID))
}
// Decrement active operation count only if operation existed // Decrement active operation count only if operation existed
hm.activeOperationCount.Add(-1) hm.activeOperationCount.Add(-1)
} else { } else {
if internal.LogLevel.DebugOrAbove() { // Debug level hm.logger().Debugf(context.Background(), logs.OperationNotTracked(connID, seqID))
internal.Logger.Printf(context.Background(), logs.OperationNotTracked(connID, seqID))
}
} }
} }
@@ -318,3 +310,10 @@ func (hm *Manager) AddNotificationHook(notificationHook NotificationHook) {
defer hm.hooksMu.Unlock() defer hm.hooksMu.Unlock()
hm.hooks = append(hm.hooks, notificationHook) hm.hooks = append(hm.hooks, notificationHook)
} }
func (hm *Manager) logger() internal.LoggerWithLevel {
if hm.config != nil && hm.config.Logger != nil {
return hm.config.Logger
}
return internal.LegacyLoggerWithLevel
}

View File

@@ -148,7 +148,7 @@ func (ph *PoolHook) OnPut(ctx context.Context, conn *pool.Conn) (shouldPool bool
if err := ph.workerManager.queueHandoff(conn); err != nil { if err := ph.workerManager.queueHandoff(conn); err != nil {
// Failed to queue handoff, remove the connection // Failed to queue handoff, remove the connection
internal.Logger.Printf(ctx, logs.FailedToQueueHandoff(conn.GetID(), err)) ph.logger().Errorf(ctx, logs.FailedToQueueHandoff(conn.GetID(), err))
// Don't pool, remove connection, no error to caller // Don't pool, remove connection, no error to caller
return false, true, nil return false, true, nil
} }
@@ -168,7 +168,7 @@ func (ph *PoolHook) OnPut(ctx context.Context, conn *pool.Conn) (shouldPool bool
// Other error - remove the connection // Other error - remove the connection
return false, true, nil return false, true, nil
} }
internal.Logger.Printf(ctx, logs.MarkedForHandoff(conn.GetID())) ph.logger().Errorf(ctx, logs.MarkedForHandoff(conn.GetID()))
return true, false, nil return true, false, nil
} }
@@ -180,3 +180,10 @@ func (ph *PoolHook) OnRemove(_ context.Context, _ *pool.Conn, _ error) {
func (ph *PoolHook) Shutdown(ctx context.Context) error { func (ph *PoolHook) Shutdown(ctx context.Context) error {
return ph.workerManager.shutdownWorkers(ctx) return ph.workerManager.shutdownWorkers(ctx)
} }
func (ph *PoolHook) logger() internal.LoggerWithLevel {
if ph.config.Logger != nil {
return ph.config.Logger
}
return internal.LegacyLoggerWithLevel
}

View File

@@ -21,13 +21,13 @@ type NotificationHandler struct {
// HandlePushNotification processes push notifications with hook support. // HandlePushNotification processes push notifications with hook support.
func (snh *NotificationHandler) HandlePushNotification(ctx context.Context, handlerCtx push.NotificationHandlerContext, notification []interface{}) error { func (snh *NotificationHandler) HandlePushNotification(ctx context.Context, handlerCtx push.NotificationHandlerContext, notification []interface{}) error {
if len(notification) == 0 { if len(notification) == 0 {
internal.Logger.Printf(ctx, logs.InvalidNotificationFormat(notification)) snh.logger().Errorf(ctx, logs.InvalidNotificationFormat(notification))
return ErrInvalidNotification return ErrInvalidNotification
} }
notificationType, ok := notification[0].(string) notificationType, ok := notification[0].(string)
if !ok { if !ok {
internal.Logger.Printf(ctx, logs.InvalidNotificationTypeFormat(notification[0])) snh.logger().Errorf(ctx, logs.InvalidNotificationTypeFormat(notification[0]))
return ErrInvalidNotification return ErrInvalidNotification
} }
@@ -64,19 +64,19 @@ func (snh *NotificationHandler) HandlePushNotification(ctx context.Context, hand
// ["MOVING", seqNum, timeS, endpoint] - per-connection handoff // ["MOVING", seqNum, timeS, endpoint] - per-connection handoff
func (snh *NotificationHandler) handleMoving(ctx context.Context, handlerCtx push.NotificationHandlerContext, notification []interface{}) error { func (snh *NotificationHandler) handleMoving(ctx context.Context, handlerCtx push.NotificationHandlerContext, notification []interface{}) error {
if len(notification) < 3 { if len(notification) < 3 {
internal.Logger.Printf(ctx, logs.InvalidNotification("MOVING", notification)) snh.logger().Errorf(ctx, logs.InvalidNotification("MOVING", notification))
return ErrInvalidNotification return ErrInvalidNotification
} }
seqID, ok := notification[1].(int64) seqID, ok := notification[1].(int64)
if !ok { if !ok {
internal.Logger.Printf(ctx, logs.InvalidSeqIDInMovingNotification(notification[1])) snh.logger().Errorf(ctx, logs.InvalidSeqIDInMovingNotification(notification[1]))
return ErrInvalidNotification return ErrInvalidNotification
} }
// Extract timeS // Extract timeS
timeS, ok := notification[2].(int64) timeS, ok := notification[2].(int64)
if !ok { if !ok {
internal.Logger.Printf(ctx, logs.InvalidTimeSInMovingNotification(notification[2])) snh.logger().Errorf(ctx, logs.InvalidTimeSInMovingNotification(notification[2]))
return ErrInvalidNotification return ErrInvalidNotification
} }
@@ -90,7 +90,7 @@ func (snh *NotificationHandler) handleMoving(ctx context.Context, handlerCtx pus
if notification[3] == nil || stringified == internal.RedisNull { if notification[3] == nil || stringified == internal.RedisNull {
newEndpoint = "" newEndpoint = ""
} else { } else {
internal.Logger.Printf(ctx, logs.InvalidNewEndpointInMovingNotification(notification[3])) snh.logger().Errorf(ctx, logs.InvalidNewEndpointInMovingNotification(notification[3]))
return ErrInvalidNotification return ErrInvalidNotification
} }
} }
@@ -99,7 +99,7 @@ func (snh *NotificationHandler) handleMoving(ctx context.Context, handlerCtx pus
// Get the connection that received this notification // Get the connection that received this notification
conn := handlerCtx.Conn conn := handlerCtx.Conn
if conn == nil { if conn == nil {
internal.Logger.Printf(ctx, logs.NoConnectionInHandlerContext("MOVING")) snh.logger().Errorf(ctx, logs.NoConnectionInHandlerContext("MOVING"))
return ErrInvalidNotification return ErrInvalidNotification
} }
@@ -108,7 +108,7 @@ func (snh *NotificationHandler) handleMoving(ctx context.Context, handlerCtx pus
if pc, ok := conn.(*pool.Conn); ok { if pc, ok := conn.(*pool.Conn); ok {
poolConn = pc poolConn = pc
} else { } else {
internal.Logger.Printf(ctx, logs.InvalidConnectionTypeInHandlerContext("MOVING", conn, handlerCtx)) snh.logger().Errorf(ctx, logs.InvalidConnectionTypeInHandlerContext("MOVING", conn, handlerCtx))
return ErrInvalidNotification return ErrInvalidNotification
} }
@@ -124,9 +124,7 @@ func (snh *NotificationHandler) handleMoving(ctx context.Context, handlerCtx pus
deadline := time.Now().Add(time.Duration(timeS) * time.Second) deadline := time.Now().Add(time.Duration(timeS) * time.Second)
// If newEndpoint is empty, we should schedule a handoff to the current endpoint in timeS/2 seconds // If newEndpoint is empty, we should schedule a handoff to the current endpoint in timeS/2 seconds
if newEndpoint == "" || newEndpoint == internal.RedisNull { if newEndpoint == "" || newEndpoint == internal.RedisNull {
if internal.LogLevel.DebugOrAbove() { snh.logger().Debugf(ctx, logs.SchedulingHandoffToCurrentEndpoint(poolConn.GetID(), float64(timeS)/2))
internal.Logger.Printf(ctx, logs.SchedulingHandoffToCurrentEndpoint(poolConn.GetID(), float64(timeS)/2))
}
// same as current endpoint // same as current endpoint
newEndpoint = snh.manager.options.GetAddr() newEndpoint = snh.manager.options.GetAddr()
// delay the handoff for timeS/2 seconds to the same endpoint // delay the handoff for timeS/2 seconds to the same endpoint
@@ -139,7 +137,7 @@ func (snh *NotificationHandler) handleMoving(ctx context.Context, handlerCtx pus
} }
if err := snh.markConnForHandoff(poolConn, newEndpoint, seqID, deadline); err != nil { if err := snh.markConnForHandoff(poolConn, newEndpoint, seqID, deadline); err != nil {
// Log error but don't fail the goroutine - use background context since original may be cancelled // Log error but don't fail the goroutine - use background context since original may be cancelled
internal.Logger.Printf(context.Background(), logs.FailedToMarkForHandoff(poolConn.GetID(), err)) snh.logger().Errorf(context.Background(), logs.FailedToMarkForHandoff(poolConn.GetID(), err))
} }
}) })
return nil return nil
@@ -150,7 +148,7 @@ func (snh *NotificationHandler) handleMoving(ctx context.Context, handlerCtx pus
func (snh *NotificationHandler) markConnForHandoff(conn *pool.Conn, newEndpoint string, seqID int64, deadline time.Time) error { func (snh *NotificationHandler) markConnForHandoff(conn *pool.Conn, newEndpoint string, seqID int64, deadline time.Time) error {
if err := conn.MarkForHandoff(newEndpoint, seqID); err != nil { if err := conn.MarkForHandoff(newEndpoint, seqID); err != nil {
internal.Logger.Printf(context.Background(), logs.FailedToMarkForHandoff(conn.GetID(), err)) snh.logger().Errorf(context.Background(), logs.FailedToMarkForHandoff(conn.GetID(), err))
// Connection is already marked for handoff, which is acceptable // Connection is already marked for handoff, which is acceptable
// This can happen if multiple MOVING notifications are received for the same connection // This can happen if multiple MOVING notifications are received for the same connection
return nil return nil
@@ -171,25 +169,23 @@ func (snh *NotificationHandler) handleMigrating(ctx context.Context, handlerCtx
// MIGRATING notifications indicate that a connection is about to be migrated // MIGRATING notifications indicate that a connection is about to be migrated
// Apply relaxed timeouts to the specific connection that received this notification // Apply relaxed timeouts to the specific connection that received this notification
if len(notification) < 2 { if len(notification) < 2 {
internal.Logger.Printf(ctx, logs.InvalidNotification("MIGRATING", notification)) snh.logger().Errorf(ctx, logs.InvalidNotification("MIGRATING", notification))
return ErrInvalidNotification return ErrInvalidNotification
} }
if handlerCtx.Conn == nil { if handlerCtx.Conn == nil {
internal.Logger.Printf(ctx, logs.NoConnectionInHandlerContext("MIGRATING")) snh.logger().Errorf(ctx, logs.NoConnectionInHandlerContext("MIGRATING"))
return ErrInvalidNotification return ErrInvalidNotification
} }
conn, ok := handlerCtx.Conn.(*pool.Conn) conn, ok := handlerCtx.Conn.(*pool.Conn)
if !ok { if !ok {
internal.Logger.Printf(ctx, logs.InvalidConnectionTypeInHandlerContext("MIGRATING", handlerCtx.Conn, handlerCtx)) snh.logger().Errorf(ctx, logs.InvalidConnectionTypeInHandlerContext("MIGRATING", handlerCtx.Conn, handlerCtx))
return ErrInvalidNotification return ErrInvalidNotification
} }
// Apply relaxed timeout to this specific connection // Apply relaxed timeout to this specific connection
if internal.LogLevel.InfoOrAbove() { snh.logger().Infof(ctx, logs.RelaxedTimeoutDueToNotification(conn.GetID(), "MIGRATING", snh.manager.config.RelaxedTimeout))
internal.Logger.Printf(ctx, logs.RelaxedTimeoutDueToNotification(conn.GetID(), "MIGRATING", snh.manager.config.RelaxedTimeout))
}
conn.SetRelaxedTimeout(snh.manager.config.RelaxedTimeout, snh.manager.config.RelaxedTimeout) conn.SetRelaxedTimeout(snh.manager.config.RelaxedTimeout, snh.manager.config.RelaxedTimeout)
return nil return nil
} }
@@ -199,26 +195,25 @@ func (snh *NotificationHandler) handleMigrated(ctx context.Context, handlerCtx p
// MIGRATED notifications indicate that a connection migration has completed // MIGRATED notifications indicate that a connection migration has completed
// Restore normal timeouts for the specific connection that received this notification // Restore normal timeouts for the specific connection that received this notification
if len(notification) < 2 { if len(notification) < 2 {
internal.Logger.Printf(ctx, logs.InvalidNotification("MIGRATED", notification)) snh.logger().Errorf(ctx, logs.InvalidNotification("MIGRATED", notification))
return ErrInvalidNotification return ErrInvalidNotification
} }
if handlerCtx.Conn == nil { if handlerCtx.Conn == nil {
internal.Logger.Printf(ctx, logs.NoConnectionInHandlerContext("MIGRATED")) snh.logger().Errorf(ctx, logs.NoConnectionInHandlerContext("MIGRATED"))
return ErrInvalidNotification return ErrInvalidNotification
} }
conn, ok := handlerCtx.Conn.(*pool.Conn) conn, ok := handlerCtx.Conn.(*pool.Conn)
if !ok { if !ok {
internal.Logger.Printf(ctx, logs.InvalidConnectionTypeInHandlerContext("MIGRATED", handlerCtx.Conn, handlerCtx)) snh.logger().Errorf(ctx, logs.InvalidConnectionTypeInHandlerContext("MIGRATED", handlerCtx.Conn, handlerCtx))
return ErrInvalidNotification return ErrInvalidNotification
} }
// Clear relaxed timeout for this specific connection // Clear relaxed timeout for this specific connection
if internal.LogLevel.InfoOrAbove() {
connID := conn.GetID() connID := conn.GetID()
internal.Logger.Printf(ctx, logs.UnrelaxedTimeout(connID)) snh.logger().Infof(ctx, logs.UnrelaxedTimeout(connID))
}
conn.ClearRelaxedTimeout() conn.ClearRelaxedTimeout()
return nil return nil
} }
@@ -228,26 +223,25 @@ func (snh *NotificationHandler) handleFailingOver(ctx context.Context, handlerCt
// FAILING_OVER notifications indicate that a connection is about to failover // FAILING_OVER notifications indicate that a connection is about to failover
// Apply relaxed timeouts to the specific connection that received this notification // Apply relaxed timeouts to the specific connection that received this notification
if len(notification) < 2 { if len(notification) < 2 {
internal.Logger.Printf(ctx, logs.InvalidNotification("FAILING_OVER", notification)) snh.logger().Errorf(ctx, logs.InvalidNotification("FAILING_OVER", notification))
return ErrInvalidNotification return ErrInvalidNotification
} }
if handlerCtx.Conn == nil { if handlerCtx.Conn == nil {
internal.Logger.Printf(ctx, logs.NoConnectionInHandlerContext("FAILING_OVER")) snh.logger().Errorf(ctx, logs.NoConnectionInHandlerContext("FAILING_OVER"))
return ErrInvalidNotification return ErrInvalidNotification
} }
conn, ok := handlerCtx.Conn.(*pool.Conn) conn, ok := handlerCtx.Conn.(*pool.Conn)
if !ok { if !ok {
internal.Logger.Printf(ctx, logs.InvalidConnectionTypeInHandlerContext("FAILING_OVER", handlerCtx.Conn, handlerCtx)) snh.logger().Errorf(ctx, logs.InvalidConnectionTypeInHandlerContext("FAILING_OVER", handlerCtx.Conn, handlerCtx))
return ErrInvalidNotification return ErrInvalidNotification
} }
// Apply relaxed timeout to this specific connection // Apply relaxed timeout to this specific connection
if internal.LogLevel.InfoOrAbove() {
connID := conn.GetID() connID := conn.GetID()
internal.Logger.Printf(ctx, logs.RelaxedTimeoutDueToNotification(connID, "FAILING_OVER", snh.manager.config.RelaxedTimeout)) snh.logger().Infof(ctx, logs.RelaxedTimeoutDueToNotification(connID, "FAILING_OVER", snh.manager.config.RelaxedTimeout))
}
conn.SetRelaxedTimeout(snh.manager.config.RelaxedTimeout, snh.manager.config.RelaxedTimeout) conn.SetRelaxedTimeout(snh.manager.config.RelaxedTimeout, snh.manager.config.RelaxedTimeout)
return nil return nil
} }
@@ -257,26 +251,31 @@ func (snh *NotificationHandler) handleFailedOver(ctx context.Context, handlerCtx
// FAILED_OVER notifications indicate that a connection failover has completed // FAILED_OVER notifications indicate that a connection failover has completed
// Restore normal timeouts for the specific connection that received this notification // Restore normal timeouts for the specific connection that received this notification
if len(notification) < 2 { if len(notification) < 2 {
internal.Logger.Printf(ctx, logs.InvalidNotification("FAILED_OVER", notification)) snh.logger().Errorf(ctx, logs.InvalidNotification("FAILED_OVER", notification))
return ErrInvalidNotification return ErrInvalidNotification
} }
if handlerCtx.Conn == nil { if handlerCtx.Conn == nil {
internal.Logger.Printf(ctx, logs.NoConnectionInHandlerContext("FAILED_OVER")) snh.logger().Errorf(ctx, logs.NoConnectionInHandlerContext("FAILED_OVER"))
return ErrInvalidNotification return ErrInvalidNotification
} }
conn, ok := handlerCtx.Conn.(*pool.Conn) conn, ok := handlerCtx.Conn.(*pool.Conn)
if !ok { if !ok {
internal.Logger.Printf(ctx, logs.InvalidConnectionTypeInHandlerContext("FAILED_OVER", handlerCtx.Conn, handlerCtx)) snh.logger().Errorf(ctx, logs.InvalidConnectionTypeInHandlerContext("FAILED_OVER", handlerCtx.Conn, handlerCtx))
return ErrInvalidNotification return ErrInvalidNotification
} }
// Clear relaxed timeout for this specific connection // Clear relaxed timeout for this specific connection
if internal.LogLevel.InfoOrAbove() {
connID := conn.GetID() connID := conn.GetID()
internal.Logger.Printf(ctx, logs.UnrelaxedTimeout(connID)) snh.logger().Infof(ctx, logs.UnrelaxedTimeout(connID))
}
conn.ClearRelaxedTimeout() conn.ClearRelaxedTimeout()
return nil return nil
} }
func (snh *NotificationHandler) logger() internal.LoggerWithLevel {
if snh.manager != nil && snh.manager.config != nil && snh.manager.config.Logger != nil {
return snh.manager.config.Logger
}
return internal.LegacyLoggerWithLevel
}

View File

@@ -14,6 +14,7 @@ import (
"time" "time"
"github.com/redis/go-redis/v9/auth" "github.com/redis/go-redis/v9/auth"
"github.com/redis/go-redis/v9/internal"
"github.com/redis/go-redis/v9/internal/pool" "github.com/redis/go-redis/v9/internal/pool"
"github.com/redis/go-redis/v9/internal/proto" "github.com/redis/go-redis/v9/internal/proto"
"github.com/redis/go-redis/v9/internal/util" "github.com/redis/go-redis/v9/internal/util"
@@ -267,6 +268,10 @@ type Options struct {
// transitions seamlessly. Requires Protocol: 3 (RESP3) for push notifications. // transitions seamlessly. Requires Protocol: 3 (RESP3) for push notifications.
// If nil, maintnotifications are in "auto" mode and will be enabled if the server supports it. // If nil, maintnotifications are in "auto" mode and will be enabled if the server supports it.
MaintNotificationsConfig *maintnotifications.Config MaintNotificationsConfig *maintnotifications.Config
// Logger is the logger used by the client for logging.
// If none is provided, the global logger [internal.LegacyLoggerWithLevel] is used.
Logger internal.LoggerWithLevel
} }
func (opt *Options) init() { func (opt *Options) init() {

View File

@@ -148,6 +148,9 @@ type ClusterOptions struct {
// If nil, maintnotifications upgrades are in "auto" mode and will be enabled if the server supports it. // If nil, maintnotifications upgrades are in "auto" mode and will be enabled if the server supports it.
// The ClusterClient does not directly work with maintnotifications, it is up to the clients in the Nodes map to work with maintnotifications. // The ClusterClient does not directly work with maintnotifications, it is up to the clients in the Nodes map to work with maintnotifications.
MaintNotificationsConfig *maintnotifications.Config MaintNotificationsConfig *maintnotifications.Config
// Logger is an optional logger for logging cluster-related messages.
Logger internal.LoggerWithLevel
} }
func (opt *ClusterOptions) init() { func (opt *ClusterOptions) init() {
@@ -390,6 +393,8 @@ func (opt *ClusterOptions) clientOptions() *Options {
UnstableResp3: opt.UnstableResp3, UnstableResp3: opt.UnstableResp3,
MaintNotificationsConfig: maintNotificationsConfig, MaintNotificationsConfig: maintNotificationsConfig,
PushNotificationProcessor: opt.PushNotificationProcessor, PushNotificationProcessor: opt.PushNotificationProcessor,
Logger: opt.Logger,
} }
} }
@@ -703,6 +708,14 @@ func (c *clusterNodes) Random() (*clusterNode, error) {
return c.GetOrCreate(addrs[n]) return c.GetOrCreate(addrs[n])
} }
func (c *clusterNodes) logger() internal.LoggerWithLevel {
if c.opt.Logger != nil {
return c.opt.Logger
} else {
return internal.LegacyLoggerWithLevel
}
}
//------------------------------------------------------------------------------ //------------------------------------------------------------------------------
type clusterSlot struct { type clusterSlot struct {
@@ -900,12 +913,12 @@ func (c *clusterState) slotClosestNode(slot int) (*clusterNode, error) {
// if all nodes are failing, we will pick the temporarily failing node with lowest latency // if all nodes are failing, we will pick the temporarily failing node with lowest latency
if minLatency < maximumNodeLatency && closestNode != nil { if minLatency < maximumNodeLatency && closestNode != nil {
internal.Logger.Printf(context.TODO(), "redis: all nodes are marked as failed, picking the temporarily failing node with lowest latency") c.nodes.logger().Errorf(context.TODO(), "redis: all nodes are marked as failed, picking the temporarily failing node with lowest latency")
return closestNode, nil return closestNode, nil
} }
// If all nodes are having the maximum latency(all pings are failing) - return a random node across the cluster // If all nodes are having the maximum latency(all pings are failing) - return a random node across the cluster
internal.Logger.Printf(context.TODO(), "redis: pings to all nodes are failing, picking a random node across the cluster") c.nodes.logger().Errorf(context.TODO(), "redis: pings to all nodes are failing, picking a random node across the cluster")
return c.nodes.Random() return c.nodes.Random()
} }
@@ -1740,7 +1753,7 @@ func (c *ClusterClient) txPipelineReadQueued(
if err := node.Client.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil { if err := node.Client.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil {
// Log the error but don't fail the command execution // Log the error but don't fail the command execution
// Push notification processing errors shouldn't break normal Redis operations // Push notification processing errors shouldn't break normal Redis operations
internal.Logger.Printf(ctx, "push: error processing pending notifications before reading reply: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before reading reply: %v", err)
} }
if err := statusCmd.readReply(rd); err != nil { if err := statusCmd.readReply(rd); err != nil {
return err return err
@@ -1751,7 +1764,7 @@ func (c *ClusterClient) txPipelineReadQueued(
if err := node.Client.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil { if err := node.Client.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil {
// Log the error but don't fail the command execution // Log the error but don't fail the command execution
// Push notification processing errors shouldn't break normal Redis operations // Push notification processing errors shouldn't break normal Redis operations
internal.Logger.Printf(ctx, "push: error processing pending notifications before reading reply: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before reading reply: %v", err)
} }
err := statusCmd.readReply(rd) err := statusCmd.readReply(rd)
if err != nil { if err != nil {
@@ -1770,7 +1783,7 @@ func (c *ClusterClient) txPipelineReadQueued(
if err := node.Client.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil { if err := node.Client.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil {
// Log the error but don't fail the command execution // Log the error but don't fail the command execution
// Push notification processing errors shouldn't break normal Redis operations // Push notification processing errors shouldn't break normal Redis operations
internal.Logger.Printf(ctx, "push: error processing pending notifications before reading reply: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before reading reply: %v", err)
} }
// Parse number of replies. // Parse number of replies.
line, err := rd.ReadLine() line, err := rd.ReadLine()
@@ -2022,13 +2035,13 @@ func (c *ClusterClient) cmdsInfo(ctx context.Context) (map[string]*CommandInfo,
func (c *ClusterClient) cmdInfo(ctx context.Context, name string) *CommandInfo { func (c *ClusterClient) cmdInfo(ctx context.Context, name string) *CommandInfo {
cmdsInfo, err := c.cmdsInfoCache.Get(ctx) cmdsInfo, err := c.cmdsInfoCache.Get(ctx)
if err != nil { if err != nil {
internal.Logger.Printf(context.TODO(), "getting command info: %s", err) c.logger().Errorf(ctx, "getting command info: %s", err)
return nil return nil
} }
info := cmdsInfo[name] info := cmdsInfo[name]
if info == nil { if info == nil {
internal.Logger.Printf(context.TODO(), "info for cmd=%s not found", name) c.logger().Errorf(ctx, "info for cmd=%s not found", name)
} }
return info return info
} }
@@ -2126,6 +2139,14 @@ func (c *ClusterClient) context(ctx context.Context) context.Context {
return context.Background() return context.Background()
} }
func (c *ClusterClient) logger() internal.LoggerWithLevel {
if c.opt.Logger != nil {
return c.opt.Logger
} else {
return internal.LegacyLoggerWithLevel
}
}
func appendIfNotExist[T comparable](vals []T, newVal T) []T { func appendIfNotExist[T comparable](vals []T, newVal T) []T {
for _, v := range vals { for _, v := range vals {
if v == newVal { if v == newVal {

View File

@@ -141,6 +141,17 @@ func mapKeys(m map[string]struct{}) []string {
return s return s
} }
// logger is a wrapper around the logger to log messages with context.
//
// it uses the client logger if set, otherwise it uses the global logger.
func (c *PubSub) logger() internal.LoggerWithLevel {
if c.opt.Logger != nil {
return c.opt.Logger
} else {
return internal.LegacyLoggerWithLevel
}
}
func (c *PubSub) _subscribe( func (c *PubSub) _subscribe(
ctx context.Context, cn *pool.Conn, redisCmd string, channels []string, ctx context.Context, cn *pool.Conn, redisCmd string, channels []string,
) error { ) error {
@@ -190,7 +201,7 @@ func (c *PubSub) reconnect(ctx context.Context, reason error) {
// Update the address in the options // Update the address in the options
oldAddr := c.cn.RemoteAddr().String() oldAddr := c.cn.RemoteAddr().String()
c.opt.Addr = newEndpoint c.opt.Addr = newEndpoint
internal.Logger.Printf(ctx, "pubsub: reconnecting to new endpoint %s (was %s)", newEndpoint, oldAddr) c.logger().Infof(ctx, "pubsub: reconnecting to new endpoint %s (was %s)", newEndpoint, oldAddr)
} }
} }
_ = c.closeTheCn(reason) _ = c.closeTheCn(reason)
@@ -475,7 +486,7 @@ func (c *PubSub) ReceiveTimeout(ctx context.Context, timeout time.Duration) (int
if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil { if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil {
// Log the error but don't fail the command execution // Log the error but don't fail the command execution
// Push notification processing errors shouldn't break normal Redis operations // Push notification processing errors shouldn't break normal Redis operations
internal.Logger.Printf(ctx, "push: conn[%d] error processing pending notifications before reading reply: %v", cn.GetID(), err) c.logger().Errorf(ctx, "push: conn[%d] error processing pending notifications before reading reply: %v", cn.GetID(), err)
} }
return c.cmd.readReply(rd) return c.cmd.readReply(rd)
}) })
@@ -634,6 +645,9 @@ func WithChannelSendTimeout(d time.Duration) ChannelOption {
type channel struct { type channel struct {
pubSub *PubSub pubSub *PubSub
// Optional logger for logging channel-related messages.
Logger internal.LoggerWithLevel
msgCh chan *Message msgCh chan *Message
allCh chan interface{} allCh chan interface{}
ping chan struct{} ping chan struct{}
@@ -733,12 +747,10 @@ func (c *channel) initMsgChan() {
<-timer.C <-timer.C
} }
case <-timer.C: case <-timer.C:
internal.Logger.Printf( c.logger().Errorf(ctx, "redis: %s channel is full for %s (message is dropped)", c, c.chanSendTimeout)
ctx, "redis: %s channel is full for %s (message is dropped)",
c, c.chanSendTimeout)
} }
default: default:
internal.Logger.Printf(ctx, "redis: unknown message type: %T", msg) c.logger().Errorf(ctx, "redis: unknown message type: %T", msg)
} }
} }
}() }()
@@ -787,13 +799,20 @@ func (c *channel) initAllChan() {
<-timer.C <-timer.C
} }
case <-timer.C: case <-timer.C:
internal.Logger.Printf( c.logger().Errorf(ctx, "redis: %s channel is full for %s (message is dropped)",
ctx, "redis: %s channel is full for %s (message is dropped)",
c, c.chanSendTimeout) c, c.chanSendTimeout)
} }
default: default:
internal.Logger.Printf(ctx, "redis: unknown message type: %T", msg) c.logger().Errorf(ctx, "redis: unknown message type: %T", msg)
} }
} }
}() }()
} }
func (c *channel) logger() internal.LoggerWithLevel {
if c.Logger != nil {
return c.Logger
} else {
return internal.LegacyLoggerWithLevel
}
}

View File

@@ -228,6 +228,9 @@ type baseClient struct {
// streamingCredentialsManager is used to manage streaming credentials // streamingCredentialsManager is used to manage streaming credentials
streamingCredentialsManager *streaming.Manager streamingCredentialsManager *streaming.Manager
// loggerWithLevel is used for logging
loggerWithLevel internal.LoggerWithLevel
} }
func (c *baseClient) clone() *baseClient { func (c *baseClient) clone() *baseClient {
@@ -242,6 +245,7 @@ func (c *baseClient) clone() *baseClient {
pushProcessor: c.pushProcessor, pushProcessor: c.pushProcessor,
maintNotificationsManager: maintNotificationsManager, maintNotificationsManager: maintNotificationsManager,
streamingCredentialsManager: c.streamingCredentialsManager, streamingCredentialsManager: c.streamingCredentialsManager,
loggerWithLevel: c.loggerWithLevel,
} }
return clone return clone
} }
@@ -336,16 +340,16 @@ func (c *baseClient) onAuthenticationErr() func(poolCn *pool.Conn, err error) {
// Close the connection to force a reconnection. // Close the connection to force a reconnection.
err := c.connPool.CloseConn(poolCn) err := c.connPool.CloseConn(poolCn)
if err != nil { if err != nil {
internal.Logger.Printf(context.Background(), "redis: failed to close connection: %v", err) c.logger().Errorf(context.Background(), "redis: failed to close connection: %v", err)
// try to close the network connection directly // try to close the network connection directly
// so that no resource is leaked // so that no resource is leaked
err := poolCn.Close() err := poolCn.Close()
if err != nil { if err != nil {
internal.Logger.Printf(context.Background(), "redis: failed to close network connection: %v", err) c.logger().Errorf(context.Background(), "redis: failed to close network connection: %v", err)
} }
} }
} }
internal.Logger.Printf(context.Background(), "redis: re-authentication failed: %v", err) c.logger().Errorf(context.Background(), "redis: re-authentication failed: %v", err)
} }
} }
} }
@@ -542,14 +546,13 @@ func (c *baseClient) initConn(ctx context.Context, cn *pool.Conn) error {
return fmt.Errorf("failed to enable maintnotifications: %w", maintNotifHandshakeErr) return fmt.Errorf("failed to enable maintnotifications: %w", maintNotifHandshakeErr)
default: // will handle auto and any other default: // will handle auto and any other
// Disabling logging here as it's too noisy. // Disabling logging here as it's too noisy.
// TODO: Enable when we have a better logging solution for log levels // c.logger().Errorf(ctx, "auto mode fallback: maintnotifications disabled due to handshake error: %v", maintNotifHandshakeErr)
// internal.Logger.Printf(ctx, "auto mode fallback: maintnotifications disabled due to handshake error: %v", maintNotifHandshakeErr)
c.opt.MaintNotificationsConfig.Mode = maintnotifications.ModeDisabled c.opt.MaintNotificationsConfig.Mode = maintnotifications.ModeDisabled
c.optLock.Unlock() c.optLock.Unlock()
// auto mode, disable maintnotifications and continue // auto mode, disable maintnotifications and continue
if initErr := c.disableMaintNotificationsUpgrades(); initErr != nil { if initErr := c.disableMaintNotificationsUpgrades(); initErr != nil {
// Log error but continue - auto mode should be resilient // Log error but continue - auto mode should be resilient
internal.Logger.Printf(ctx, "failed to disable maintnotifications in auto mode: %v", initErr) c.logger().Errorf(ctx, "failed to disable maintnotifications in auto mode: %v", initErr)
} }
} }
} else { } else {
@@ -613,7 +616,7 @@ func (c *baseClient) releaseConn(ctx context.Context, cn *pool.Conn, err error)
} else { } else {
// process any pending push notifications before returning the connection to the pool // process any pending push notifications before returning the connection to the pool
if err := c.processPushNotifications(ctx, cn); err != nil { if err := c.processPushNotifications(ctx, cn); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before releasing connection: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before releasing connection: %v", err)
} }
c.connPool.Put(ctx, cn) c.connPool.Put(ctx, cn)
} }
@@ -680,7 +683,7 @@ func (c *baseClient) _process(ctx context.Context, cmd Cmder, attempt int) (bool
if err := c.withConn(ctx, func(ctx context.Context, cn *pool.Conn) error { if err := c.withConn(ctx, func(ctx context.Context, cn *pool.Conn) error {
// Process any pending push notifications before executing the command // Process any pending push notifications before executing the command
if err := c.processPushNotifications(ctx, cn); err != nil { if err := c.processPushNotifications(ctx, cn); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before command: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before command: %v", err)
} }
if err := cn.WithWriter(c.context(ctx), c.opt.WriteTimeout, func(wr *proto.Writer) error { if err := cn.WithWriter(c.context(ctx), c.opt.WriteTimeout, func(wr *proto.Writer) error {
@@ -703,7 +706,7 @@ func (c *baseClient) _process(ctx context.Context, cmd Cmder, attempt int) (bool
if err := cn.WithReader(c.context(ctx), c.cmdTimeout(cmd), func(rd *proto.Reader) error { if err := cn.WithReader(c.context(ctx), c.cmdTimeout(cmd), func(rd *proto.Reader) error {
// To be sure there are no buffered push notifications, we process them before reading the reply // To be sure there are no buffered push notifications, we process them before reading the reply
if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil { if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before reading reply: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before reading reply: %v", err)
} }
return readReplyFunc(rd) return readReplyFunc(rd)
}); err != nil { }); err != nil {
@@ -749,6 +752,16 @@ func (c *baseClient) context(ctx context.Context) context.Context {
return context.Background() return context.Background()
} }
// logger is a wrapper around the logger to log messages with context.
// it uses the client logger if set, otherwise it uses the global logger.
func (c *baseClient) logger() internal.LoggerWithLevel {
if c.opt.Logger != nil {
return c.opt.Logger
} else {
return internal.LegacyLoggerWithLevel
}
}
// createInitConnFunc creates a connection initialization function that can be used for reconnections. // createInitConnFunc creates a connection initialization function that can be used for reconnections.
func (c *baseClient) createInitConnFunc() func(context.Context, *pool.Conn) error { func (c *baseClient) createInitConnFunc() func(context.Context, *pool.Conn) error {
return func(ctx context.Context, cn *pool.Conn) error { return func(ctx context.Context, cn *pool.Conn) error {
@@ -860,7 +873,7 @@ func (c *baseClient) generalProcessPipeline(
lastErr = c.withConn(ctx, func(ctx context.Context, cn *pool.Conn) error { lastErr = c.withConn(ctx, func(ctx context.Context, cn *pool.Conn) error {
// Process any pending push notifications before executing the pipeline // Process any pending push notifications before executing the pipeline
if err := c.processPushNotifications(ctx, cn); err != nil { if err := c.processPushNotifications(ctx, cn); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before processing pipeline: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before processing pipeline: %v", err)
} }
var err error var err error
canRetry, err = p(ctx, cn, cmds) canRetry, err = p(ctx, cn, cmds)
@@ -882,7 +895,7 @@ func (c *baseClient) pipelineProcessCmds(
) (bool, error) { ) (bool, error) {
// Process any pending push notifications before executing the pipeline // Process any pending push notifications before executing the pipeline
if err := c.processPushNotifications(ctx, cn); err != nil { if err := c.processPushNotifications(ctx, cn); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before writing pipeline: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before writing pipeline: %v", err)
} }
if err := cn.WithWriter(c.context(ctx), c.opt.WriteTimeout, func(wr *proto.Writer) error { if err := cn.WithWriter(c.context(ctx), c.opt.WriteTimeout, func(wr *proto.Writer) error {
@@ -906,7 +919,7 @@ func (c *baseClient) pipelineReadCmds(ctx context.Context, cn *pool.Conn, rd *pr
for i, cmd := range cmds { for i, cmd := range cmds {
// To be sure there are no buffered push notifications, we process them before reading the reply // To be sure there are no buffered push notifications, we process them before reading the reply
if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil { if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before reading reply: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before reading reply: %v", err)
} }
err := cmd.readReply(rd) err := cmd.readReply(rd)
cmd.SetErr(err) cmd.SetErr(err)
@@ -924,7 +937,7 @@ func (c *baseClient) txPipelineProcessCmds(
) (bool, error) { ) (bool, error) {
// Process any pending push notifications before executing the transaction pipeline // Process any pending push notifications before executing the transaction pipeline
if err := c.processPushNotifications(ctx, cn); err != nil { if err := c.processPushNotifications(ctx, cn); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before transaction: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before transaction: %v", err)
} }
if err := cn.WithWriter(c.context(ctx), c.opt.WriteTimeout, func(wr *proto.Writer) error { if err := cn.WithWriter(c.context(ctx), c.opt.WriteTimeout, func(wr *proto.Writer) error {
@@ -958,7 +971,7 @@ func (c *baseClient) txPipelineProcessCmds(
func (c *baseClient) txPipelineReadQueued(ctx context.Context, cn *pool.Conn, rd *proto.Reader, statusCmd *StatusCmd, cmds []Cmder) error { func (c *baseClient) txPipelineReadQueued(ctx context.Context, cn *pool.Conn, rd *proto.Reader, statusCmd *StatusCmd, cmds []Cmder) error {
// To be sure there are no buffered push notifications, we process them before reading the reply // To be sure there are no buffered push notifications, we process them before reading the reply
if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil { if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before reading reply: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before reading reply: %v", err)
} }
// Parse +OK. // Parse +OK.
if err := statusCmd.readReply(rd); err != nil { if err := statusCmd.readReply(rd); err != nil {
@@ -969,7 +982,7 @@ func (c *baseClient) txPipelineReadQueued(ctx context.Context, cn *pool.Conn, rd
for _, cmd := range cmds { for _, cmd := range cmds {
// To be sure there are no buffered push notifications, we process them before reading the reply // To be sure there are no buffered push notifications, we process them before reading the reply
if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil { if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before reading reply: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before reading reply: %v", err)
} }
if err := statusCmd.readReply(rd); err != nil { if err := statusCmd.readReply(rd); err != nil {
cmd.SetErr(err) cmd.SetErr(err)
@@ -981,7 +994,7 @@ func (c *baseClient) txPipelineReadQueued(ctx context.Context, cn *pool.Conn, rd
// To be sure there are no buffered push notifications, we process them before reading the reply // To be sure there are no buffered push notifications, we process them before reading the reply
if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil { if err := c.processPendingPushNotificationWithReader(ctx, cn, rd); err != nil {
internal.Logger.Printf(ctx, "push: error processing pending notifications before reading reply: %v", err) c.logger().Errorf(ctx, "push: error processing pending notifications before reading reply: %v", err)
} }
// Parse number of replies. // Parse number of replies.
line, err := rd.ReadLine() line, err := rd.ReadLine()
@@ -1055,7 +1068,7 @@ func NewClient(opt *Options) *Client {
if opt.MaintNotificationsConfig != nil && opt.MaintNotificationsConfig.Mode != maintnotifications.ModeDisabled && opt.Protocol == 3 { if opt.MaintNotificationsConfig != nil && opt.MaintNotificationsConfig.Mode != maintnotifications.ModeDisabled && opt.Protocol == 3 {
err := c.enableMaintNotificationsUpgrades() err := c.enableMaintNotificationsUpgrades()
if err != nil { if err != nil {
internal.Logger.Printf(context.Background(), "failed to initialize maintnotifications: %v", err) c.logger().Errorf(context.Background(), "failed to initialize maintnotifications: %v", err)
if opt.MaintNotificationsConfig.Mode == maintnotifications.ModeEnabled { if opt.MaintNotificationsConfig.Mode == maintnotifications.ModeEnabled {
/* /*
Design decision: panic here to fail fast if maintnotifications cannot be enabled when explicitly requested. Design decision: panic here to fail fast if maintnotifications cannot be enabled when explicitly requested.

13
ring.go
View File

@@ -154,6 +154,8 @@ type RingOptions struct {
DisableIdentity bool DisableIdentity bool
IdentitySuffix string IdentitySuffix string
UnstableResp3 bool UnstableResp3 bool
Logger internal.LoggerWithLevel
} }
func (opt *RingOptions) init() { func (opt *RingOptions) init() {
@@ -345,7 +347,7 @@ func (c *ringSharding) SetAddrs(addrs map[string]string) {
cleanup := func(shards map[string]*ringShard) { cleanup := func(shards map[string]*ringShard) {
for addr, shard := range shards { for addr, shard := range shards {
if err := shard.Client.Close(); err != nil { if err := shard.Client.Close(); err != nil {
internal.Logger.Printf(context.Background(), "shard.Close %s failed: %s", addr, err) c.logger().Errorf(context.Background(), "shard.Close %s failed: %s", addr, err)
} }
} }
} }
@@ -490,7 +492,7 @@ func (c *ringSharding) Heartbeat(ctx context.Context, frequency time.Duration) {
for _, shard := range c.List() { for _, shard := range c.List() {
isUp := c.opt.HeartbeatFn(ctx, shard.Client) isUp := c.opt.HeartbeatFn(ctx, shard.Client)
if shard.Vote(isUp) { if shard.Vote(isUp) {
internal.Logger.Printf(ctx, "ring shard state changed: %s", shard) c.logger().Infof(ctx, "ring shard state changed: %s", shard)
rebalance = true rebalance = true
} }
} }
@@ -559,6 +561,13 @@ func (c *ringSharding) Close() error {
return firstErr return firstErr
} }
func (c *ringSharding) logger() internal.LoggerWithLevel {
if c.opt.Logger != nil {
return c.opt.Logger
}
return internal.LegacyLoggerWithLevel
}
//------------------------------------------------------------------------------ //------------------------------------------------------------------------------
// Ring is a Redis client that uses consistent hashing to distribute // Ring is a Redis client that uses consistent hashing to distribute

View File

@@ -149,6 +149,9 @@ type FailoverOptions struct {
// If nil, maintnotifications upgrades are disabled. // If nil, maintnotifications upgrades are disabled.
// (however if Mode is nil, it defaults to "auto" - enable if server supports it) // (however if Mode is nil, it defaults to "auto" - enable if server supports it)
//MaintNotificationsConfig *maintnotifications.Config //MaintNotificationsConfig *maintnotifications.Config
// Optional logger for logging
Logger internal.LoggerWithLevel
} }
func (opt *FailoverOptions) clientOptions() *Options { func (opt *FailoverOptions) clientOptions() *Options {
@@ -199,6 +202,8 @@ func (opt *FailoverOptions) clientOptions() *Options {
MaintNotificationsConfig: &maintnotifications.Config{ MaintNotificationsConfig: &maintnotifications.Config{
Mode: maintnotifications.ModeDisabled, Mode: maintnotifications.ModeDisabled,
}, },
Logger: opt.Logger,
} }
} }
@@ -247,6 +252,8 @@ func (opt *FailoverOptions) sentinelOptions(addr string) *Options {
MaintNotificationsConfig: &maintnotifications.Config{ MaintNotificationsConfig: &maintnotifications.Config{
Mode: maintnotifications.ModeDisabled, Mode: maintnotifications.ModeDisabled,
}, },
Logger: opt.Logger,
} }
} }
@@ -300,6 +307,8 @@ func (opt *FailoverOptions) clusterOptions() *ClusterOptions {
MaintNotificationsConfig: &maintnotifications.Config{ MaintNotificationsConfig: &maintnotifications.Config{
Mode: maintnotifications.ModeDisabled, Mode: maintnotifications.ModeDisabled,
}, },
Logger: opt.Logger,
} }
} }
@@ -831,7 +840,7 @@ func (c *sentinelFailover) MasterAddr(ctx context.Context) (string, error) {
return "", err return "", err
} }
// Continue on other errors // Continue on other errors
internal.Logger.Printf(ctx, "sentinel: GetMasterAddrByName name=%q failed: %s", c.logger().Errorf(ctx, "sentinel: GetMasterAddrByName name=%q failed: %s",
c.opt.MasterName, err) c.opt.MasterName, err)
} else { } else {
return addr, nil return addr, nil
@@ -849,7 +858,7 @@ func (c *sentinelFailover) MasterAddr(ctx context.Context) (string, error) {
return "", err return "", err
} }
// Continue on other errors // Continue on other errors
internal.Logger.Printf(ctx, "sentinel: GetMasterAddrByName name=%q failed: %s", c.logger().Errorf(ctx, "sentinel: GetMasterAddrByName name=%q failed: %s",
c.opt.MasterName, err) c.opt.MasterName, err)
} else { } else {
return addr, nil return addr, nil
@@ -878,7 +887,7 @@ func (c *sentinelFailover) MasterAddr(ctx context.Context) (string, error) {
sentinelCli := NewSentinelClient(c.opt.sentinelOptions(addr)) sentinelCli := NewSentinelClient(c.opt.sentinelOptions(addr))
addrVal, err := sentinelCli.GetMasterAddrByName(ctx, c.opt.MasterName).Result() addrVal, err := sentinelCli.GetMasterAddrByName(ctx, c.opt.MasterName).Result()
if err != nil { if err != nil {
internal.Logger.Printf(ctx, "sentinel: GetMasterAddrByName addr=%s, master=%q failed: %s", c.logger().Errorf(ctx, "sentinel: GetMasterAddrByName addr=%s, master=%q failed: %s",
addr, c.opt.MasterName, err) addr, c.opt.MasterName, err)
_ = sentinelCli.Close() _ = sentinelCli.Close()
errCh <- err errCh <- err
@@ -889,7 +898,7 @@ func (c *sentinelFailover) MasterAddr(ctx context.Context) (string, error) {
// Push working sentinel to the top // Push working sentinel to the top
c.sentinelAddrs[0], c.sentinelAddrs[i] = c.sentinelAddrs[i], c.sentinelAddrs[0] c.sentinelAddrs[0], c.sentinelAddrs[i] = c.sentinelAddrs[i], c.sentinelAddrs[0]
c.setSentinel(ctx, sentinelCli) c.setSentinel(ctx, sentinelCli)
internal.Logger.Printf(ctx, "sentinel: selected addr=%s masterAddr=%s", addr, masterAddr) c.logger().Infof(ctx, "sentinel: selected addr=%s masterAddr=%s", addr, masterAddr)
cancel() cancel()
}) })
}(i, sentinelAddr) }(i, sentinelAddr)
@@ -934,7 +943,7 @@ func (c *sentinelFailover) replicaAddrs(ctx context.Context, useDisconnected boo
return nil, err return nil, err
} }
// Continue on other errors // Continue on other errors
internal.Logger.Printf(ctx, "sentinel: Replicas name=%q failed: %s", c.logger().Errorf(ctx, "sentinel: Replicas name=%q failed: %s",
c.opt.MasterName, err) c.opt.MasterName, err)
} else if len(addrs) > 0 { } else if len(addrs) > 0 {
return addrs, nil return addrs, nil
@@ -952,7 +961,7 @@ func (c *sentinelFailover) replicaAddrs(ctx context.Context, useDisconnected boo
return nil, err return nil, err
} }
// Continue on other errors // Continue on other errors
internal.Logger.Printf(ctx, "sentinel: Replicas name=%q failed: %s", c.logger().Errorf(ctx, "sentinel: Replicas name=%q failed: %s",
c.opt.MasterName, err) c.opt.MasterName, err)
} else if len(addrs) > 0 { } else if len(addrs) > 0 {
return addrs, nil return addrs, nil
@@ -973,7 +982,7 @@ func (c *sentinelFailover) replicaAddrs(ctx context.Context, useDisconnected boo
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return nil, err return nil, err
} }
internal.Logger.Printf(ctx, "sentinel: Replicas master=%q failed: %s", c.logger().Errorf(ctx, "sentinel: Replicas master=%q failed: %s",
c.opt.MasterName, err) c.opt.MasterName, err)
continue continue
} }
@@ -1006,7 +1015,7 @@ func (c *sentinelFailover) getMasterAddr(ctx context.Context, sentinel *Sentinel
func (c *sentinelFailover) getReplicaAddrs(ctx context.Context, sentinel *SentinelClient) ([]string, error) { func (c *sentinelFailover) getReplicaAddrs(ctx context.Context, sentinel *SentinelClient) ([]string, error) {
addrs, err := sentinel.Replicas(ctx, c.opt.MasterName).Result() addrs, err := sentinel.Replicas(ctx, c.opt.MasterName).Result()
if err != nil { if err != nil {
internal.Logger.Printf(ctx, "sentinel: Replicas name=%q failed: %s", c.logger().Errorf(ctx, "sentinel: Replicas name=%q failed: %s",
c.opt.MasterName, err) c.opt.MasterName, err)
return nil, err return nil, err
} }
@@ -1054,7 +1063,7 @@ func (c *sentinelFailover) trySwitchMaster(ctx context.Context, addr string) {
} }
c.masterAddr = addr c.masterAddr = addr
internal.Logger.Printf(ctx, "sentinel: new master=%q addr=%q", c.logger().Infof(ctx, "sentinel: new master=%q addr=%q",
c.opt.MasterName, addr) c.opt.MasterName, addr)
if c.onFailover != nil { if c.onFailover != nil {
c.onFailover(ctx, addr) c.onFailover(ctx, addr)
@@ -1075,7 +1084,7 @@ func (c *sentinelFailover) setSentinel(ctx context.Context, sentinel *SentinelCl
func (c *sentinelFailover) discoverSentinels(ctx context.Context) { func (c *sentinelFailover) discoverSentinels(ctx context.Context) {
sentinels, err := c.sentinel.Sentinels(ctx, c.opt.MasterName).Result() sentinels, err := c.sentinel.Sentinels(ctx, c.opt.MasterName).Result()
if err != nil { if err != nil {
internal.Logger.Printf(ctx, "sentinel: Sentinels master=%q failed: %s", c.opt.MasterName, err) c.logger().Errorf(ctx, "sentinel: Sentinels master=%q failed: %s", c.opt.MasterName, err)
return return
} }
for _, sentinel := range sentinels { for _, sentinel := range sentinels {
@@ -1090,7 +1099,7 @@ func (c *sentinelFailover) discoverSentinels(ctx context.Context) {
if ip != "" && port != "" { if ip != "" && port != "" {
sentinelAddr := net.JoinHostPort(ip, port) sentinelAddr := net.JoinHostPort(ip, port)
if !contains(c.sentinelAddrs, sentinelAddr) { if !contains(c.sentinelAddrs, sentinelAddr) {
internal.Logger.Printf(ctx, "sentinel: discovered new sentinel=%q for master=%q", c.logger().Infof(ctx, "sentinel: discovered new sentinel=%q for master=%q",
sentinelAddr, c.opt.MasterName) sentinelAddr, c.opt.MasterName)
c.sentinelAddrs = append(c.sentinelAddrs, sentinelAddr) c.sentinelAddrs = append(c.sentinelAddrs, sentinelAddr)
} }
@@ -1110,7 +1119,7 @@ func (c *sentinelFailover) listen(pubsub *PubSub) {
if msg.Channel == "+switch-master" { if msg.Channel == "+switch-master" {
parts := strings.Split(msg.Payload, " ") parts := strings.Split(msg.Payload, " ")
if parts[0] != c.opt.MasterName { if parts[0] != c.opt.MasterName {
internal.Logger.Printf(pubsub.getContext(), "sentinel: ignore addr for master=%q", parts[0]) c.logger().Infof(pubsub.getContext(), "sentinel: ignore addr for master=%q", parts[0])
continue continue
} }
addr := net.JoinHostPort(parts[3], parts[4]) addr := net.JoinHostPort(parts[3], parts[4])
@@ -1123,6 +1132,13 @@ func (c *sentinelFailover) listen(pubsub *PubSub) {
} }
} }
func (c *sentinelFailover) logger() internal.LoggerWithLevel {
if c.opt.Logger != nil {
return c.opt.Logger
}
return internal.LegacyLoggerWithLevel
}
func contains(slice []string, str string) bool { func contains(slice []string, str string) bool {
for _, s := range slice { for _, s := range slice {
if s == str { if s == str {