mirror of
				https://github.com/redis/go-redis.git
				synced 2025-11-04 02:33:24 +03:00 
			
		
		
		
	feat(pool): Improve success rate of new connections (#3518)
* async create conn * update default values and testcase * fix comments * fix data race * remove context.WithoutCancel, which is a function introduced in Go 1.21 * fix TestDialerRetryConfiguration/DefaultDialerRetries, because tryDial are likely done in async flow * change to share failed to delivery connection to other waiting * remove chinese comment * fix: optimize WantConnQueue benchmarks to prevent memory exhaustion - Fix BenchmarkWantConnQueue_Dequeue timeout issue by limiting pre-population - Use object pooling in BenchmarkWantConnQueue_Enqueue to reduce allocations - Optimize BenchmarkWantConnQueue_EnqueueDequeue with reusable wantConn pool - Prevent GitHub Actions benchmark failures due to excessive memory usage Before: BenchmarkWantConnQueue_Dequeue ran for 11+ minutes and was killed After: All benchmarks complete in ~8 seconds with consistent performance * format * fix turn leaks --------- Co-authored-by: Nedyalko Dyakov <1547186+ndyakov@users.noreply.github.com> Co-authored-by: Hristo Temelski <hristo.temelski@redis.com>
This commit is contained in:
		@@ -54,6 +54,7 @@ func TestEventDrivenHandoffIntegration(t *testing.T) {
 | 
			
		||||
				return &mockNetConn{addr: "original:6379"}, nil
 | 
			
		||||
			},
 | 
			
		||||
			PoolSize:           int32(5),
 | 
			
		||||
			MaxConcurrentDials: 5,
 | 
			
		||||
			PoolTimeout:        time.Second,
 | 
			
		||||
		})
 | 
			
		||||
 | 
			
		||||
@@ -154,6 +155,7 @@ func TestEventDrivenHandoffIntegration(t *testing.T) {
 | 
			
		||||
			},
 | 
			
		||||
 | 
			
		||||
			PoolSize:           int32(10),
 | 
			
		||||
			MaxConcurrentDials: 10,
 | 
			
		||||
			PoolTimeout:        time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
@@ -226,6 +228,7 @@ func TestEventDrivenHandoffIntegration(t *testing.T) {
 | 
			
		||||
			},
 | 
			
		||||
 | 
			
		||||
			PoolSize:           int32(3),
 | 
			
		||||
			MaxConcurrentDials: 3,
 | 
			
		||||
			PoolTimeout:        time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
@@ -289,6 +292,7 @@ func TestEventDrivenHandoffIntegration(t *testing.T) {
 | 
			
		||||
			},
 | 
			
		||||
 | 
			
		||||
			PoolSize:           int32(2),
 | 
			
		||||
			MaxConcurrentDials: 2,
 | 
			
		||||
			PoolTimeout:        time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 
 | 
			
		||||
@@ -33,6 +33,7 @@ func BenchmarkPoolGetPut(b *testing.B) {
 | 
			
		||||
			connPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
				Dialer:             dummyDialer,
 | 
			
		||||
				PoolSize:           int32(bm.poolSize),
 | 
			
		||||
				MaxConcurrentDials: bm.poolSize,
 | 
			
		||||
				PoolTimeout:        time.Second,
 | 
			
		||||
				DialTimeout:        1 * time.Second,
 | 
			
		||||
				ConnMaxIdleTime:    time.Hour,
 | 
			
		||||
@@ -77,6 +78,7 @@ func BenchmarkPoolGetRemove(b *testing.B) {
 | 
			
		||||
			connPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
				Dialer:             dummyDialer,
 | 
			
		||||
				PoolSize:           int32(bm.poolSize),
 | 
			
		||||
				MaxConcurrentDials: bm.poolSize,
 | 
			
		||||
				PoolTimeout:        time.Second,
 | 
			
		||||
				DialTimeout:        1 * time.Second,
 | 
			
		||||
				ConnMaxIdleTime:    time.Hour,
 | 
			
		||||
 
 | 
			
		||||
@@ -26,6 +26,7 @@ var _ = Describe("Buffer Size Configuration", func() {
 | 
			
		||||
		connPool = pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           int32(1),
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			PoolTimeout:        1000,
 | 
			
		||||
		})
 | 
			
		||||
 | 
			
		||||
@@ -48,6 +49,7 @@ var _ = Describe("Buffer Size Configuration", func() {
 | 
			
		||||
		connPool = pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           int32(1),
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			PoolTimeout:        1000,
 | 
			
		||||
			ReadBufferSize:     customReadSize,
 | 
			
		||||
			WriteBufferSize:    customWriteSize,
 | 
			
		||||
@@ -69,6 +71,7 @@ var _ = Describe("Buffer Size Configuration", func() {
 | 
			
		||||
		connPool = pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           int32(1),
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			PoolTimeout:        1000,
 | 
			
		||||
			ReadBufferSize:     0, // Should use default
 | 
			
		||||
			WriteBufferSize:    0, // Should use default
 | 
			
		||||
@@ -105,6 +108,7 @@ var _ = Describe("Buffer Size Configuration", func() {
 | 
			
		||||
		connPool = pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           int32(1),
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			PoolTimeout:        1000,
 | 
			
		||||
			// ReadBufferSize and WriteBufferSize are not set (will be 0)
 | 
			
		||||
		})
 | 
			
		||||
 
 | 
			
		||||
@@ -192,6 +192,7 @@ func TestPoolWithHooks(t *testing.T) {
 | 
			
		||||
			return &net.TCPConn{}, nil // Mock connection
 | 
			
		||||
		},
 | 
			
		||||
		PoolSize:           1,
 | 
			
		||||
		MaxConcurrentDials: 1,
 | 
			
		||||
		DialTimeout:        time.Second,
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
 
 | 
			
		||||
@@ -98,6 +98,7 @@ type Options struct {
 | 
			
		||||
 | 
			
		||||
	PoolFIFO                 bool
 | 
			
		||||
	PoolSize                 int32
 | 
			
		||||
	MaxConcurrentDials       int
 | 
			
		||||
	DialTimeout              time.Duration
 | 
			
		||||
	PoolTimeout              time.Duration
 | 
			
		||||
	MinIdleConns             int32
 | 
			
		||||
@@ -127,6 +128,8 @@ type ConnPool struct {
 | 
			
		||||
	lastDialError atomic.Value
 | 
			
		||||
 | 
			
		||||
	queue           chan struct{}
 | 
			
		||||
	dialsInProgress chan struct{}
 | 
			
		||||
	dialsQueue      *wantConnQueue
 | 
			
		||||
 | 
			
		||||
	connsMu   sync.Mutex
 | 
			
		||||
	conns     map[uint64]*Conn
 | 
			
		||||
@@ -154,6 +157,8 @@ func NewConnPool(opt *Options) *ConnPool {
 | 
			
		||||
 | 
			
		||||
		queue:           make(chan struct{}, opt.PoolSize),
 | 
			
		||||
		conns:           make(map[uint64]*Conn),
 | 
			
		||||
		dialsInProgress: make(chan struct{}, opt.MaxConcurrentDials),
 | 
			
		||||
		dialsQueue:      newWantConnQueue(),
 | 
			
		||||
		idleConns:       make([]*Conn, 0, opt.PoolSize),
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
@@ -233,6 +238,7 @@ func (p *ConnPool) checkMinIdleConns() {
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (p *ConnPool) addIdleConn() error {
 | 
			
		||||
@@ -491,9 +497,8 @@ func (p *ConnPool) getConn(ctx context.Context) (*Conn, error) {
 | 
			
		||||
 | 
			
		||||
	atomic.AddUint32(&p.stats.Misses, 1)
 | 
			
		||||
 | 
			
		||||
	newcn, err := p.newConn(ctx, true)
 | 
			
		||||
	newcn, err := p.queuedNewConn(ctx)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		p.freeTurn()
 | 
			
		||||
		return nil, err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
@@ -512,6 +517,99 @@ func (p *ConnPool) getConn(ctx context.Context) (*Conn, error) {
 | 
			
		||||
	return newcn, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (p *ConnPool) queuedNewConn(ctx context.Context) (*Conn, error) {
 | 
			
		||||
	select {
 | 
			
		||||
	case p.dialsInProgress <- struct{}{}:
 | 
			
		||||
		// Got permission, proceed to create connection
 | 
			
		||||
	case <-ctx.Done():
 | 
			
		||||
		p.freeTurn()
 | 
			
		||||
		return nil, ctx.Err()
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	dialCtx, cancel := context.WithTimeout(context.Background(), p.cfg.DialTimeout)
 | 
			
		||||
 | 
			
		||||
	w := &wantConn{
 | 
			
		||||
		ctx:       dialCtx,
 | 
			
		||||
		cancelCtx: cancel,
 | 
			
		||||
		result:    make(chan wantConnResult, 1),
 | 
			
		||||
	}
 | 
			
		||||
	var err error
 | 
			
		||||
	defer func() {
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			if cn := w.cancel(); cn != nil {
 | 
			
		||||
				p.putIdleConn(ctx, cn)
 | 
			
		||||
				p.freeTurn()
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
	}()
 | 
			
		||||
 | 
			
		||||
	p.dialsQueue.enqueue(w)
 | 
			
		||||
 | 
			
		||||
	go func(w *wantConn) {
 | 
			
		||||
		var freeTurnCalled bool
 | 
			
		||||
		defer func() {
 | 
			
		||||
			if err := recover(); err != nil {
 | 
			
		||||
				if !freeTurnCalled {
 | 
			
		||||
					p.freeTurn()
 | 
			
		||||
				}
 | 
			
		||||
				internal.Logger.Printf(context.Background(), "queuedNewConn panic: %+v", err)
 | 
			
		||||
			}
 | 
			
		||||
		}()
 | 
			
		||||
 | 
			
		||||
		defer w.cancelCtx()
 | 
			
		||||
		defer func() { <-p.dialsInProgress }() // Release connection creation permission
 | 
			
		||||
 | 
			
		||||
		dialCtx := w.getCtxForDial()
 | 
			
		||||
		cn, cnErr := p.newConn(dialCtx, true)
 | 
			
		||||
		delivered := w.tryDeliver(cn, cnErr)
 | 
			
		||||
		if cnErr == nil && delivered {
 | 
			
		||||
			return
 | 
			
		||||
		} else if cnErr == nil && !delivered {
 | 
			
		||||
			p.putIdleConn(dialCtx, cn)
 | 
			
		||||
			p.freeTurn()
 | 
			
		||||
			freeTurnCalled = true
 | 
			
		||||
		} else {
 | 
			
		||||
			p.freeTurn()
 | 
			
		||||
			freeTurnCalled = true
 | 
			
		||||
		}
 | 
			
		||||
	}(w)
 | 
			
		||||
 | 
			
		||||
	select {
 | 
			
		||||
	case <-ctx.Done():
 | 
			
		||||
		err = ctx.Err()
 | 
			
		||||
		return nil, err
 | 
			
		||||
	case result := <-w.result:
 | 
			
		||||
		err = result.err
 | 
			
		||||
		return result.cn, err
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (p *ConnPool) putIdleConn(ctx context.Context, cn *Conn) {
 | 
			
		||||
	for {
 | 
			
		||||
		w, ok := p.dialsQueue.dequeue()
 | 
			
		||||
		if !ok {
 | 
			
		||||
			break
 | 
			
		||||
		}
 | 
			
		||||
		if w.tryDeliver(cn, nil) {
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	cn.SetUsable(true)
 | 
			
		||||
 | 
			
		||||
	p.connsMu.Lock()
 | 
			
		||||
	defer p.connsMu.Unlock()
 | 
			
		||||
 | 
			
		||||
	if p.closed() {
 | 
			
		||||
		_ = cn.Close()
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// poolSize is increased in newConn
 | 
			
		||||
	p.idleConns = append(p.idleConns, cn)
 | 
			
		||||
	p.idleConnsLen.Add(1)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (p *ConnPool) waitTurn(ctx context.Context) error {
 | 
			
		||||
	select {
 | 
			
		||||
	case <-ctx.Done():
 | 
			
		||||
 
 | 
			
		||||
@@ -3,6 +3,7 @@ package pool_test
 | 
			
		||||
import (
 | 
			
		||||
	"context"
 | 
			
		||||
	"errors"
 | 
			
		||||
	"fmt"
 | 
			
		||||
	"net"
 | 
			
		||||
	"sync"
 | 
			
		||||
	"sync/atomic"
 | 
			
		||||
@@ -23,6 +24,7 @@ var _ = Describe("ConnPool", func() {
 | 
			
		||||
		connPool = pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           int32(10),
 | 
			
		||||
			MaxConcurrentDials: 10,
 | 
			
		||||
			PoolTimeout:        time.Hour,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			ConnMaxIdleTime:    time.Millisecond,
 | 
			
		||||
@@ -48,6 +50,7 @@ var _ = Describe("ConnPool", func() {
 | 
			
		||||
				return &net.TCPConn{}, nil
 | 
			
		||||
			},
 | 
			
		||||
			PoolSize:           int32(10),
 | 
			
		||||
			MaxConcurrentDials: 10,
 | 
			
		||||
			PoolTimeout:        time.Hour,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			ConnMaxIdleTime:    time.Millisecond,
 | 
			
		||||
@@ -57,7 +60,7 @@ var _ = Describe("ConnPool", func() {
 | 
			
		||||
		Expect(connPool.Close()).NotTo(HaveOccurred())
 | 
			
		||||
		close(closedChan)
 | 
			
		||||
 | 
			
		||||
		// We wait for 1 second and believe that checkMinIdleConns has been executed.
 | 
			
		||||
		// We wait for 1 second and believe that checkIdleConns has been executed.
 | 
			
		||||
		time.Sleep(time.Second)
 | 
			
		||||
 | 
			
		||||
		Expect(connPool.Stats()).To(Equal(&pool.Stats{
 | 
			
		||||
@@ -133,6 +136,7 @@ var _ = Describe("MinIdleConns", func() {
 | 
			
		||||
		connPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           int32(poolSize),
 | 
			
		||||
			MaxConcurrentDials: poolSize,
 | 
			
		||||
			MinIdleConns:       int32(minIdleConns),
 | 
			
		||||
			PoolTimeout:        100 * time.Millisecond,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
@@ -312,6 +316,7 @@ var _ = Describe("race", func() {
 | 
			
		||||
		connPool = pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           int32(10),
 | 
			
		||||
			MaxConcurrentDials: 10,
 | 
			
		||||
			PoolTimeout:        time.Minute,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			ConnMaxIdleTime:    time.Millisecond,
 | 
			
		||||
@@ -342,6 +347,7 @@ var _ = Describe("race", func() {
 | 
			
		||||
				return &net.TCPConn{}, nil
 | 
			
		||||
			},
 | 
			
		||||
			PoolSize:           int32(1000),
 | 
			
		||||
			MaxConcurrentDials: 1000,
 | 
			
		||||
			MinIdleConns:       int32(50),
 | 
			
		||||
			PoolTimeout:        3 * time.Second,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
@@ -369,6 +375,7 @@ var _ = Describe("race", func() {
 | 
			
		||||
				panic("test panic")
 | 
			
		||||
			},
 | 
			
		||||
			PoolSize:           int32(100),
 | 
			
		||||
			MaxConcurrentDials: 100,
 | 
			
		||||
			MinIdleConns:       int32(30),
 | 
			
		||||
		}
 | 
			
		||||
		p := pool.NewConnPool(opt)
 | 
			
		||||
@@ -387,6 +394,7 @@ var _ = Describe("race", func() {
 | 
			
		||||
				return &net.TCPConn{}, nil
 | 
			
		||||
			},
 | 
			
		||||
			PoolSize:           int32(1),
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			PoolTimeout:        3 * time.Second,
 | 
			
		||||
		}
 | 
			
		||||
		p := pool.NewConnPool(opt)
 | 
			
		||||
@@ -418,6 +426,7 @@ var _ = Describe("race", func() {
 | 
			
		||||
				return &net.TCPConn{}, nil
 | 
			
		||||
			},
 | 
			
		||||
			PoolSize:           int32(1),
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			PoolTimeout:        testPoolTimeout,
 | 
			
		||||
		}
 | 
			
		||||
		p := pool.NewConnPool(opt)
 | 
			
		||||
@@ -452,6 +461,7 @@ func TestDialerRetryConfiguration(t *testing.T) {
 | 
			
		||||
		connPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             failingDialer,
 | 
			
		||||
			PoolSize:           1,
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			PoolTimeout:        time.Second,
 | 
			
		||||
			DialTimeout:        time.Second,
 | 
			
		||||
			DialerRetries:      3,                     // Custom retry count
 | 
			
		||||
@@ -485,6 +495,7 @@ func TestDialerRetryConfiguration(t *testing.T) {
 | 
			
		||||
		connPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             failingDialer,
 | 
			
		||||
			PoolSize:           1,
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			PoolTimeout:        time.Second,
 | 
			
		||||
			DialTimeout:        time.Second,
 | 
			
		||||
			// DialerRetries and DialerRetryTimeout not set - should use defaults
 | 
			
		||||
@@ -509,6 +520,525 @@ func TestDialerRetryConfiguration(t *testing.T) {
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
var _ = Describe("queuedNewConn", func() {
 | 
			
		||||
	ctx := context.Background()
 | 
			
		||||
 | 
			
		||||
	It("should successfully create connection when pool is exhausted", func() {
 | 
			
		||||
		testPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           1,
 | 
			
		||||
			MaxConcurrentDials: 2,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			PoolTimeout:        2 * time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// Fill the pool
 | 
			
		||||
		conn1, err := testPool.Get(ctx)
 | 
			
		||||
		Expect(err).NotTo(HaveOccurred())
 | 
			
		||||
		Expect(conn1).NotTo(BeNil())
 | 
			
		||||
 | 
			
		||||
		// Get second connection in another goroutine
 | 
			
		||||
		done := make(chan struct{})
 | 
			
		||||
		var conn2 *pool.Conn
 | 
			
		||||
		var err2 error
 | 
			
		||||
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer GinkgoRecover()
 | 
			
		||||
			conn2, err2 = testPool.Get(ctx)
 | 
			
		||||
			close(done)
 | 
			
		||||
		}()
 | 
			
		||||
 | 
			
		||||
		// Wait a bit to let the second Get start waiting
 | 
			
		||||
		time.Sleep(100 * time.Millisecond)
 | 
			
		||||
 | 
			
		||||
		// Release first connection to let second Get acquire Turn
 | 
			
		||||
		testPool.Put(ctx, conn1)
 | 
			
		||||
 | 
			
		||||
		// Wait for second Get to complete
 | 
			
		||||
		<-done
 | 
			
		||||
		Expect(err2).NotTo(HaveOccurred())
 | 
			
		||||
		Expect(conn2).NotTo(BeNil())
 | 
			
		||||
 | 
			
		||||
		// Clean up second connection
 | 
			
		||||
		testPool.Put(ctx, conn2)
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	It("should handle context cancellation before acquiring dialsInProgress", func() {
 | 
			
		||||
		slowDialer := func(ctx context.Context) (net.Conn, error) {
 | 
			
		||||
			// Simulate slow dialing to let first connection creation occupy dialsInProgress
 | 
			
		||||
			time.Sleep(200 * time.Millisecond)
 | 
			
		||||
			return newDummyConn(), nil
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		testPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             slowDialer,
 | 
			
		||||
			PoolSize:           2,
 | 
			
		||||
			MaxConcurrentDials: 1, // Limit to 1 so second request cannot get dialsInProgress permission
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			PoolTimeout:        1 * time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// Start first connection creation, this will occupy dialsInProgress
 | 
			
		||||
		done1 := make(chan struct{})
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer GinkgoRecover()
 | 
			
		||||
			conn1, err := testPool.Get(ctx)
 | 
			
		||||
			if err == nil {
 | 
			
		||||
				defer testPool.Put(ctx, conn1)
 | 
			
		||||
			}
 | 
			
		||||
			close(done1)
 | 
			
		||||
		}()
 | 
			
		||||
 | 
			
		||||
		// Wait a bit to ensure first request starts and occupies dialsInProgress
 | 
			
		||||
		time.Sleep(50 * time.Millisecond)
 | 
			
		||||
 | 
			
		||||
		// Create a context that will be cancelled quickly
 | 
			
		||||
		cancelCtx, cancel := context.WithTimeout(ctx, 100*time.Millisecond)
 | 
			
		||||
		defer cancel()
 | 
			
		||||
 | 
			
		||||
		// Second request should timeout while waiting for dialsInProgress
 | 
			
		||||
		_, err := testPool.Get(cancelCtx)
 | 
			
		||||
		Expect(err).To(Equal(context.DeadlineExceeded))
 | 
			
		||||
 | 
			
		||||
		// Wait for first request to complete
 | 
			
		||||
		<-done1
 | 
			
		||||
 | 
			
		||||
		// Verify all turns are released after requests complete
 | 
			
		||||
		Eventually(func() int {
 | 
			
		||||
			return testPool.QueueLen()
 | 
			
		||||
		}, "1s", "50ms").Should(Equal(0), "All turns should be released after requests complete")
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	It("should handle context cancellation while waiting for connection result", func() {
 | 
			
		||||
		// This test focuses on proper error handling when context is cancelled
 | 
			
		||||
		// during queuedNewConn execution (not testing connection reuse)
 | 
			
		||||
 | 
			
		||||
		slowDialer := func(ctx context.Context) (net.Conn, error) {
 | 
			
		||||
			// Simulate slow dialing
 | 
			
		||||
			time.Sleep(500 * time.Millisecond)
 | 
			
		||||
			return newDummyConn(), nil
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		testPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             slowDialer,
 | 
			
		||||
			PoolSize:           1,
 | 
			
		||||
			MaxConcurrentDials: 2,
 | 
			
		||||
			DialTimeout:        2 * time.Second,
 | 
			
		||||
			PoolTimeout:        2 * time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// Get first connection to fill the pool
 | 
			
		||||
		conn1, err := testPool.Get(ctx)
 | 
			
		||||
		Expect(err).NotTo(HaveOccurred())
 | 
			
		||||
 | 
			
		||||
		// Create a context that will be cancelled during connection creation
 | 
			
		||||
		cancelCtx, cancel := context.WithTimeout(ctx, 200*time.Millisecond)
 | 
			
		||||
		defer cancel()
 | 
			
		||||
 | 
			
		||||
		// This request should timeout while waiting for connection creation result
 | 
			
		||||
		// Testing the error handling path in queuedNewConn select statement
 | 
			
		||||
		done := make(chan struct{})
 | 
			
		||||
		var err2 error
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer GinkgoRecover()
 | 
			
		||||
			_, err2 = testPool.Get(cancelCtx)
 | 
			
		||||
			close(done)
 | 
			
		||||
		}()
 | 
			
		||||
 | 
			
		||||
		<-done
 | 
			
		||||
		Expect(err2).To(Equal(context.DeadlineExceeded))
 | 
			
		||||
 | 
			
		||||
		// Verify turn state - background goroutine may still hold turn
 | 
			
		||||
		// Note: Background connection creation will complete and release turn
 | 
			
		||||
		Eventually(func() int {
 | 
			
		||||
			return testPool.QueueLen()
 | 
			
		||||
		}, "1s", "50ms").Should(Equal(1), "Only conn1's turn should be held")
 | 
			
		||||
 | 
			
		||||
		// Clean up - release the first connection
 | 
			
		||||
		testPool.Put(ctx, conn1)
 | 
			
		||||
 | 
			
		||||
		// Verify all turns are released after cleanup
 | 
			
		||||
		Eventually(func() int {
 | 
			
		||||
			return testPool.QueueLen()
 | 
			
		||||
		}, "1s", "50ms").Should(Equal(0), "All turns should be released after cleanup")
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	It("should handle dial failures gracefully", func() {
 | 
			
		||||
		alwaysFailDialer := func(ctx context.Context) (net.Conn, error) {
 | 
			
		||||
			return nil, fmt.Errorf("dial failed")
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		testPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             alwaysFailDialer,
 | 
			
		||||
			PoolSize:           1,
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			PoolTimeout:        1 * time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// This call should fail, testing error handling branch in goroutine
 | 
			
		||||
		_, err := testPool.Get(ctx)
 | 
			
		||||
		Expect(err).To(HaveOccurred())
 | 
			
		||||
		Expect(err.Error()).To(ContainSubstring("dial failed"))
 | 
			
		||||
 | 
			
		||||
		// Verify turn is released after dial failure
 | 
			
		||||
		Eventually(func() int {
 | 
			
		||||
			return testPool.QueueLen()
 | 
			
		||||
		}, "1s", "50ms").Should(Equal(0), "Turn should be released after dial failure")
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	It("should handle connection creation success with normal delivery", func() {
 | 
			
		||||
		// This test verifies normal case where connection creation and delivery both succeed
 | 
			
		||||
		testPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           1,
 | 
			
		||||
			MaxConcurrentDials: 2,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			PoolTimeout:        2 * time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// Get first connection
 | 
			
		||||
		conn1, err := testPool.Get(ctx)
 | 
			
		||||
		Expect(err).NotTo(HaveOccurred())
 | 
			
		||||
 | 
			
		||||
		// Get second connection in another goroutine
 | 
			
		||||
		done := make(chan struct{})
 | 
			
		||||
		var conn2 *pool.Conn
 | 
			
		||||
		var err2 error
 | 
			
		||||
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer GinkgoRecover()
 | 
			
		||||
			conn2, err2 = testPool.Get(ctx)
 | 
			
		||||
			close(done)
 | 
			
		||||
		}()
 | 
			
		||||
 | 
			
		||||
		// Wait a bit to let second Get start waiting
 | 
			
		||||
		time.Sleep(100 * time.Millisecond)
 | 
			
		||||
 | 
			
		||||
		// Release first connection
 | 
			
		||||
		testPool.Put(ctx, conn1)
 | 
			
		||||
 | 
			
		||||
		// Wait for second Get to complete
 | 
			
		||||
		<-done
 | 
			
		||||
		Expect(err2).NotTo(HaveOccurred())
 | 
			
		||||
		Expect(conn2).NotTo(BeNil())
 | 
			
		||||
 | 
			
		||||
		// Clean up second connection
 | 
			
		||||
		testPool.Put(ctx, conn2)
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	It("should handle MaxConcurrentDials limit", func() {
 | 
			
		||||
		testPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             dummyDialer,
 | 
			
		||||
			PoolSize:           3,
 | 
			
		||||
			MaxConcurrentDials: 1, // Only allow 1 concurrent dial
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			PoolTimeout:        1 * time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// Get all connections to fill the pool
 | 
			
		||||
		var conns []*pool.Conn
 | 
			
		||||
		for i := 0; i < 3; i++ {
 | 
			
		||||
			conn, err := testPool.Get(ctx)
 | 
			
		||||
			Expect(err).NotTo(HaveOccurred())
 | 
			
		||||
			conns = append(conns, conn)
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Now pool is full, next request needs to create new connection
 | 
			
		||||
		// But due to MaxConcurrentDials=1, only one concurrent dial is allowed
 | 
			
		||||
		done := make(chan struct{})
 | 
			
		||||
		var err4 error
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer GinkgoRecover()
 | 
			
		||||
			_, err4 = testPool.Get(ctx)
 | 
			
		||||
			close(done)
 | 
			
		||||
		}()
 | 
			
		||||
 | 
			
		||||
		// Release one connection to let the request complete
 | 
			
		||||
		time.Sleep(100 * time.Millisecond)
 | 
			
		||||
		testPool.Put(ctx, conns[0])
 | 
			
		||||
 | 
			
		||||
		<-done
 | 
			
		||||
		Expect(err4).NotTo(HaveOccurred())
 | 
			
		||||
 | 
			
		||||
		// Clean up remaining connections
 | 
			
		||||
		for i := 1; i < len(conns); i++ {
 | 
			
		||||
			testPool.Put(ctx, conns[i])
 | 
			
		||||
		}
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	It("should reuse connections created in background after request timeout", func() {
 | 
			
		||||
		// This test focuses on connection reuse mechanism:
 | 
			
		||||
		// When a request times out but background connection creation succeeds,
 | 
			
		||||
		// the created connection should be added to pool for future reuse
 | 
			
		||||
 | 
			
		||||
		slowDialer := func(ctx context.Context) (net.Conn, error) {
 | 
			
		||||
			// Simulate delay for connection creation
 | 
			
		||||
			time.Sleep(100 * time.Millisecond)
 | 
			
		||||
			return newDummyConn(), nil
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		testPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             slowDialer,
 | 
			
		||||
			PoolSize:           1,
 | 
			
		||||
			MaxConcurrentDials: 1,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			PoolTimeout:        150 * time.Millisecond, // Short timeout for waiting Turn
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// Fill the pool with one connection
 | 
			
		||||
		conn1, err := testPool.Get(ctx)
 | 
			
		||||
		Expect(err).NotTo(HaveOccurred())
 | 
			
		||||
		// Don't put it back yet, so pool is full
 | 
			
		||||
 | 
			
		||||
		// Start a goroutine that will create a new connection but take time
 | 
			
		||||
		done1 := make(chan struct{})
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer GinkgoRecover()
 | 
			
		||||
			defer close(done1)
 | 
			
		||||
			// This will trigger queuedNewConn since pool is full
 | 
			
		||||
			conn, err := testPool.Get(ctx)
 | 
			
		||||
			if err == nil {
 | 
			
		||||
				// Put connection back to pool after creation
 | 
			
		||||
				time.Sleep(50 * time.Millisecond)
 | 
			
		||||
				testPool.Put(ctx, conn)
 | 
			
		||||
			}
 | 
			
		||||
		}()
 | 
			
		||||
 | 
			
		||||
		// Wait a bit to let the goroutine start and begin connection creation
 | 
			
		||||
		time.Sleep(50 * time.Millisecond)
 | 
			
		||||
 | 
			
		||||
		// Now make a request that should timeout waiting for Turn
 | 
			
		||||
		start := time.Now()
 | 
			
		||||
		_, err = testPool.Get(ctx)
 | 
			
		||||
		duration := time.Since(start)
 | 
			
		||||
 | 
			
		||||
		Expect(err).To(Equal(pool.ErrPoolTimeout))
 | 
			
		||||
		// Should timeout around PoolTimeout
 | 
			
		||||
		Expect(duration).To(BeNumerically("~", 150*time.Millisecond, 50*time.Millisecond))
 | 
			
		||||
 | 
			
		||||
		// Release the first connection to allow the background creation to complete
 | 
			
		||||
		testPool.Put(ctx, conn1)
 | 
			
		||||
 | 
			
		||||
		// Wait for background connection creation to complete
 | 
			
		||||
		<-done1
 | 
			
		||||
		time.Sleep(100 * time.Millisecond)
 | 
			
		||||
 | 
			
		||||
		// CORE TEST: Verify connection reuse mechanism
 | 
			
		||||
		// The connection created in background should now be available in pool
 | 
			
		||||
		start = time.Now()
 | 
			
		||||
		conn3, err := testPool.Get(ctx)
 | 
			
		||||
		duration = time.Since(start)
 | 
			
		||||
 | 
			
		||||
		Expect(err).NotTo(HaveOccurred())
 | 
			
		||||
		Expect(conn3).NotTo(BeNil())
 | 
			
		||||
		// Should be fast since connection is from pool (not newly created)
 | 
			
		||||
		Expect(duration).To(BeNumerically("<", 50*time.Millisecond))
 | 
			
		||||
 | 
			
		||||
		testPool.Put(ctx, conn3)
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	It("recover queuedNewConn panic", func() {
 | 
			
		||||
		opt := &pool.Options{
 | 
			
		||||
			Dialer: func(ctx context.Context) (net.Conn, error) {
 | 
			
		||||
				panic("test panic in queuedNewConn")
 | 
			
		||||
			},
 | 
			
		||||
			PoolSize:           int32(10),
 | 
			
		||||
			MaxConcurrentDials: 10,
 | 
			
		||||
			DialTimeout:        1 * time.Second,
 | 
			
		||||
			PoolTimeout:        1 * time.Second,
 | 
			
		||||
		}
 | 
			
		||||
		testPool := pool.NewConnPool(opt)
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// Trigger queuedNewConn - calling Get() on empty pool will trigger it
 | 
			
		||||
		// Since dialer will panic, it should be handled by recover
 | 
			
		||||
		ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
 | 
			
		||||
		defer cancel()
 | 
			
		||||
 | 
			
		||||
		// Try to get connections multiple times, each will trigger panic but should be properly recovered
 | 
			
		||||
		for i := 0; i < 3; i++ {
 | 
			
		||||
			conn, err := testPool.Get(ctx)
 | 
			
		||||
			// Connection should be nil, error should exist (panic converted to error)
 | 
			
		||||
			Expect(conn).To(BeNil())
 | 
			
		||||
			Expect(err).To(HaveOccurred())
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Verify state after panic recovery:
 | 
			
		||||
		// - turn should be properly released (QueueLen() == 0)
 | 
			
		||||
		// - connection counts should be correct (TotalConns == 0, IdleConns == 0)
 | 
			
		||||
		Eventually(func() bool {
 | 
			
		||||
			stats := testPool.Stats()
 | 
			
		||||
			queueLen := testPool.QueueLen()
 | 
			
		||||
			return stats.TotalConns == 0 && stats.IdleConns == 0 && queueLen == 0
 | 
			
		||||
		}, "3s", "50ms").Should(BeTrue())
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	It("should handle connection creation success but delivery failure (putIdleConn path)", func() {
 | 
			
		||||
		// This test covers the most important untested branch in queuedNewConn:
 | 
			
		||||
		// cnErr == nil && !delivered -> putIdleConn()
 | 
			
		||||
 | 
			
		||||
		// Use slow dialer to ensure request times out before connection is ready
 | 
			
		||||
		slowDialer := func(ctx context.Context) (net.Conn, error) {
 | 
			
		||||
			// Delay long enough for client request to timeout first
 | 
			
		||||
			time.Sleep(300 * time.Millisecond)
 | 
			
		||||
			return newDummyConn(), nil
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		testPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             slowDialer,
 | 
			
		||||
			PoolSize:           1,
 | 
			
		||||
			MaxConcurrentDials: 2,
 | 
			
		||||
			DialTimeout:        500 * time.Millisecond, // Long enough for dialer to complete
 | 
			
		||||
			PoolTimeout:        100 * time.Millisecond, // Client requests will timeout quickly
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// Record initial idle connection count
 | 
			
		||||
		initialIdleConns := testPool.Stats().IdleConns
 | 
			
		||||
 | 
			
		||||
		// Make a request that will timeout
 | 
			
		||||
		// This request will start queuedNewConn, create connection, but fail to deliver due to timeout
 | 
			
		||||
		shortCtx, cancel := context.WithTimeout(context.Background(), 150*time.Millisecond)
 | 
			
		||||
		defer cancel()
 | 
			
		||||
 | 
			
		||||
		conn, err := testPool.Get(shortCtx)
 | 
			
		||||
 | 
			
		||||
		// Request should fail due to timeout
 | 
			
		||||
		Expect(err).To(HaveOccurred())
 | 
			
		||||
		Expect(conn).To(BeNil())
 | 
			
		||||
 | 
			
		||||
		// However, background queuedNewConn should continue and complete connection creation
 | 
			
		||||
		// Since it cannot deliver (request timed out), it should call putIdleConn to add connection to idle pool
 | 
			
		||||
		Eventually(func() bool {
 | 
			
		||||
			stats := testPool.Stats()
 | 
			
		||||
			return stats.IdleConns > initialIdleConns
 | 
			
		||||
		}, "1s", "50ms").Should(BeTrue())
 | 
			
		||||
 | 
			
		||||
		// Verify the connection can indeed be used by subsequent requests
 | 
			
		||||
		conn2, err2 := testPool.Get(context.Background())
 | 
			
		||||
		Expect(err2).NotTo(HaveOccurred())
 | 
			
		||||
		Expect(conn2).NotTo(BeNil())
 | 
			
		||||
		Expect(conn2.IsUsable()).To(BeTrue())
 | 
			
		||||
 | 
			
		||||
		// Cleanup
 | 
			
		||||
		testPool.Put(context.Background(), conn2)
 | 
			
		||||
 | 
			
		||||
		// Verify turn is released after putIdleConn path completes
 | 
			
		||||
		// This is critical: ensures freeTurn() was called in the putIdleConn branch
 | 
			
		||||
		Eventually(func() int {
 | 
			
		||||
			return testPool.QueueLen()
 | 
			
		||||
		}, "1s", "50ms").Should(Equal(0),
 | 
			
		||||
			"Turn should be released after putIdleConn path completes")
 | 
			
		||||
	})
 | 
			
		||||
 | 
			
		||||
	It("should not leak turn when delivering connection via putIdleConn", func() {
 | 
			
		||||
		// This test verifies that freeTurn() is called when putIdleConn successfully
 | 
			
		||||
		// delivers a connection to another waiting request
 | 
			
		||||
		//
 | 
			
		||||
		// Scenario:
 | 
			
		||||
		// 1. Request A: timeout 150ms, connection creation takes 200ms
 | 
			
		||||
		// 2. Request B: timeout 500ms, connection creation takes 400ms
 | 
			
		||||
		// 3. Both requests enter dialsQueue and start async connection creation
 | 
			
		||||
		// 4. Request A times out at 150ms
 | 
			
		||||
		// 5. Request A's connection completes at 200ms
 | 
			
		||||
		// 6. putIdleConn delivers Request A's connection to Request B
 | 
			
		||||
		// 7. queuedNewConn must call freeTurn()
 | 
			
		||||
		// 8. Check: QueueLen should be 1 (only B holding turn), not 2 (A's turn leaked)
 | 
			
		||||
 | 
			
		||||
		callCount := int32(0)
 | 
			
		||||
 | 
			
		||||
		controlledDialer := func(ctx context.Context) (net.Conn, error) {
 | 
			
		||||
			count := atomic.AddInt32(&callCount, 1)
 | 
			
		||||
			if count == 1 {
 | 
			
		||||
				// Request A's connection: takes 200ms
 | 
			
		||||
				time.Sleep(200 * time.Millisecond)
 | 
			
		||||
			} else {
 | 
			
		||||
				// Request B's connection: takes 400ms (longer, so A's connection is used)
 | 
			
		||||
				time.Sleep(400 * time.Millisecond)
 | 
			
		||||
			}
 | 
			
		||||
			return newDummyConn(), nil
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		testPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
			Dialer:             controlledDialer,
 | 
			
		||||
			PoolSize:           2, // Allows both requests to get turns
 | 
			
		||||
			MaxConcurrentDials: 2, // Allows both connections to be created simultaneously
 | 
			
		||||
			DialTimeout:        500 * time.Millisecond,
 | 
			
		||||
			PoolTimeout:        1 * time.Second,
 | 
			
		||||
		})
 | 
			
		||||
		defer testPool.Close()
 | 
			
		||||
 | 
			
		||||
		// Verify initial state
 | 
			
		||||
		Expect(testPool.QueueLen()).To(Equal(0))
 | 
			
		||||
 | 
			
		||||
		// Request A: Short timeout (150ms), connection takes 200ms
 | 
			
		||||
		reqADone := make(chan error, 1)
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer GinkgoRecover()
 | 
			
		||||
			shortCtx, cancel := context.WithTimeout(ctx, 150*time.Millisecond)
 | 
			
		||||
			defer cancel()
 | 
			
		||||
			_, err := testPool.Get(shortCtx)
 | 
			
		||||
			reqADone <- err
 | 
			
		||||
		}()
 | 
			
		||||
 | 
			
		||||
		// Wait for Request A to acquire turn and enter dialsQueue
 | 
			
		||||
		time.Sleep(50 * time.Millisecond)
 | 
			
		||||
		Expect(testPool.QueueLen()).To(Equal(1), "Request A should occupy turn")
 | 
			
		||||
 | 
			
		||||
		// Request B: Long timeout (500ms), will receive Request A's connection
 | 
			
		||||
		reqBDone := make(chan struct{})
 | 
			
		||||
		var reqBConn *pool.Conn
 | 
			
		||||
		var reqBErr error
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer GinkgoRecover()
 | 
			
		||||
			longCtx, cancel := context.WithTimeout(ctx, 500*time.Millisecond)
 | 
			
		||||
			defer cancel()
 | 
			
		||||
			reqBConn, reqBErr = testPool.Get(longCtx)
 | 
			
		||||
			close(reqBDone)
 | 
			
		||||
		}()
 | 
			
		||||
 | 
			
		||||
		// Wait for Request B to acquire turn and enter dialsQueue
 | 
			
		||||
		time.Sleep(50 * time.Millisecond)
 | 
			
		||||
		Expect(testPool.QueueLen()).To(Equal(2), "Both requests should occupy turns")
 | 
			
		||||
 | 
			
		||||
		// Request A times out at 150ms
 | 
			
		||||
		reqAErr := <-reqADone
 | 
			
		||||
		Expect(reqAErr).To(HaveOccurred(), "Request A should timeout")
 | 
			
		||||
 | 
			
		||||
		// Request A's connection completes at 200ms
 | 
			
		||||
		// putIdleConn delivers it to Request B via tryDeliver
 | 
			
		||||
		// queuedNewConn MUST call freeTurn() to release Request A's turn
 | 
			
		||||
		<-reqBDone
 | 
			
		||||
		Expect(reqBErr).NotTo(HaveOccurred(), "Request B should receive Request A's connection")
 | 
			
		||||
		Expect(reqBConn).NotTo(BeNil())
 | 
			
		||||
 | 
			
		||||
		// CRITICAL CHECK: Turn leak detection
 | 
			
		||||
		// After Request B receives connection from putIdleConn:
 | 
			
		||||
		// - Request A's turn SHOULD be released (via freeTurn)
 | 
			
		||||
		// - Request B's turn is still held (will release on Put)
 | 
			
		||||
		// Expected QueueLen: 1 (only Request B)
 | 
			
		||||
		// If Bug exists (missing freeTurn): QueueLen: 2 (Request A's turn leaked)
 | 
			
		||||
		time.Sleep(100 * time.Millisecond) // Allow time for turn release
 | 
			
		||||
		currentQueueLen := testPool.QueueLen()
 | 
			
		||||
 | 
			
		||||
		Expect(currentQueueLen).To(Equal(1),
 | 
			
		||||
			"QueueLen should be 1 (only Request B holding turn). "+
 | 
			
		||||
				"If it's 2, Request A's turn leaked due to missing freeTurn()")
 | 
			
		||||
 | 
			
		||||
		// Cleanup
 | 
			
		||||
		testPool.Put(ctx, reqBConn)
 | 
			
		||||
		Eventually(func() int { return testPool.QueueLen() }, "500ms").Should(Equal(0))
 | 
			
		||||
	})
 | 
			
		||||
})
 | 
			
		||||
 | 
			
		||||
func init() {
 | 
			
		||||
	logging.Disable()
 | 
			
		||||
}
 | 
			
		||||
 
 | 
			
		||||
							
								
								
									
										93
									
								
								internal/pool/want_conn.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										93
									
								
								internal/pool/want_conn.go
									
									
									
									
									
										Normal file
									
								
							@@ -0,0 +1,93 @@
 | 
			
		||||
package pool
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"context"
 | 
			
		||||
	"sync"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
type wantConn struct {
 | 
			
		||||
	mu        sync.Mutex      // protects ctx, done and sending of the result
 | 
			
		||||
	ctx       context.Context // context for dial, cleared after delivered or canceled
 | 
			
		||||
	cancelCtx context.CancelFunc
 | 
			
		||||
	done      bool                // true after delivered or canceled
 | 
			
		||||
	result    chan wantConnResult // channel to deliver connection or error
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// getCtxForDial returns context for dial or nil if connection was delivered or canceled.
 | 
			
		||||
func (w *wantConn) getCtxForDial() context.Context {
 | 
			
		||||
	w.mu.Lock()
 | 
			
		||||
	defer w.mu.Unlock()
 | 
			
		||||
 | 
			
		||||
	return w.ctx
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (w *wantConn) tryDeliver(cn *Conn, err error) bool {
 | 
			
		||||
	w.mu.Lock()
 | 
			
		||||
	defer w.mu.Unlock()
 | 
			
		||||
	if w.done {
 | 
			
		||||
		return false
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	w.done = true
 | 
			
		||||
	w.ctx = nil
 | 
			
		||||
 | 
			
		||||
	w.result <- wantConnResult{cn: cn, err: err}
 | 
			
		||||
	close(w.result)
 | 
			
		||||
 | 
			
		||||
	return true
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (w *wantConn) cancel() *Conn {
 | 
			
		||||
	w.mu.Lock()
 | 
			
		||||
	var cn *Conn
 | 
			
		||||
	if w.done {
 | 
			
		||||
		select {
 | 
			
		||||
		case result := <-w.result:
 | 
			
		||||
			cn = result.cn
 | 
			
		||||
		default:
 | 
			
		||||
		}
 | 
			
		||||
	} else {
 | 
			
		||||
		close(w.result)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	w.done = true
 | 
			
		||||
	w.ctx = nil
 | 
			
		||||
	w.mu.Unlock()
 | 
			
		||||
 | 
			
		||||
	return cn
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type wantConnResult struct {
 | 
			
		||||
	cn  *Conn
 | 
			
		||||
	err error
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type wantConnQueue struct {
 | 
			
		||||
	mu    sync.RWMutex
 | 
			
		||||
	items []*wantConn
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func newWantConnQueue() *wantConnQueue {
 | 
			
		||||
	return &wantConnQueue{
 | 
			
		||||
		items: make([]*wantConn, 0),
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (q *wantConnQueue) enqueue(w *wantConn) {
 | 
			
		||||
	q.mu.Lock()
 | 
			
		||||
	defer q.mu.Unlock()
 | 
			
		||||
	q.items = append(q.items, w)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (q *wantConnQueue) dequeue() (*wantConn, bool) {
 | 
			
		||||
	q.mu.Lock()
 | 
			
		||||
	defer q.mu.Unlock()
 | 
			
		||||
 | 
			
		||||
	if len(q.items) == 0 {
 | 
			
		||||
		return nil, false
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	item := q.items[0]
 | 
			
		||||
	q.items = q.items[1:]
 | 
			
		||||
	return item, true
 | 
			
		||||
}
 | 
			
		||||
							
								
								
									
										444
									
								
								internal/pool/want_conn_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										444
									
								
								internal/pool/want_conn_test.go
									
									
									
									
									
										Normal file
									
								
							@@ -0,0 +1,444 @@
 | 
			
		||||
package pool
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"context"
 | 
			
		||||
	"errors"
 | 
			
		||||
	"sync"
 | 
			
		||||
	"testing"
 | 
			
		||||
	"time"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
func TestWantConn_getCtxForDial(t *testing.T) {
 | 
			
		||||
	ctx := context.Background()
 | 
			
		||||
	w := &wantConn{
 | 
			
		||||
		ctx:    ctx,
 | 
			
		||||
		result: make(chan wantConnResult, 1),
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Test getting context when not done
 | 
			
		||||
	gotCtx := w.getCtxForDial()
 | 
			
		||||
	if gotCtx != ctx {
 | 
			
		||||
		t.Errorf("getCtxForDial() = %v, want %v", gotCtx, ctx)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Test getting context when done
 | 
			
		||||
	w.done = true
 | 
			
		||||
	w.ctx = nil
 | 
			
		||||
	gotCtx = w.getCtxForDial()
 | 
			
		||||
	if gotCtx != nil {
 | 
			
		||||
		t.Errorf("getCtxForDial() after done = %v, want nil", gotCtx)
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestWantConn_tryDeliver_Success(t *testing.T) {
 | 
			
		||||
	w := &wantConn{
 | 
			
		||||
		ctx:    context.Background(),
 | 
			
		||||
		result: make(chan wantConnResult, 1),
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Create a mock connection
 | 
			
		||||
	conn := &Conn{}
 | 
			
		||||
 | 
			
		||||
	// Test successful delivery
 | 
			
		||||
	delivered := w.tryDeliver(conn, nil)
 | 
			
		||||
	if !delivered {
 | 
			
		||||
		t.Error("tryDeliver() = false, want true")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check that wantConn is marked as done
 | 
			
		||||
	if !w.done {
 | 
			
		||||
		t.Error("wantConn.done = false, want true after delivery")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check that context is cleared
 | 
			
		||||
	if w.ctx != nil {
 | 
			
		||||
		t.Error("wantConn.ctx should be nil after delivery")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check that result is sent
 | 
			
		||||
	select {
 | 
			
		||||
	case result := <-w.result:
 | 
			
		||||
		if result.cn != conn {
 | 
			
		||||
			t.Errorf("result.cn = %v, want %v", result.cn, conn)
 | 
			
		||||
		}
 | 
			
		||||
		if result.err != nil {
 | 
			
		||||
			t.Errorf("result.err = %v, want nil", result.err)
 | 
			
		||||
		}
 | 
			
		||||
	case <-time.After(time.Millisecond):
 | 
			
		||||
		t.Error("Expected result to be sent to channel")
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestWantConn_tryDeliver_WithError(t *testing.T) {
 | 
			
		||||
	w := &wantConn{
 | 
			
		||||
		ctx:    context.Background(),
 | 
			
		||||
		result: make(chan wantConnResult, 1),
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	testErr := errors.New("test error")
 | 
			
		||||
 | 
			
		||||
	// Test delivery with error
 | 
			
		||||
	delivered := w.tryDeliver(nil, testErr)
 | 
			
		||||
	if !delivered {
 | 
			
		||||
		t.Error("tryDeliver() = false, want true")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check result
 | 
			
		||||
	select {
 | 
			
		||||
	case result := <-w.result:
 | 
			
		||||
		if result.cn != nil {
 | 
			
		||||
			t.Errorf("result.cn = %v, want nil", result.cn)
 | 
			
		||||
		}
 | 
			
		||||
		if result.err != testErr {
 | 
			
		||||
			t.Errorf("result.err = %v, want %v", result.err, testErr)
 | 
			
		||||
		}
 | 
			
		||||
	case <-time.After(time.Millisecond):
 | 
			
		||||
		t.Error("Expected result to be sent to channel")
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestWantConn_tryDeliver_AlreadyDone(t *testing.T) {
 | 
			
		||||
	w := &wantConn{
 | 
			
		||||
		ctx:    context.Background(),
 | 
			
		||||
		done:   true, // Already done
 | 
			
		||||
		result: make(chan wantConnResult, 1),
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Test delivery when already done
 | 
			
		||||
	delivered := w.tryDeliver(&Conn{}, nil)
 | 
			
		||||
	if delivered {
 | 
			
		||||
		t.Error("tryDeliver() = true, want false when already done")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check that no result is sent
 | 
			
		||||
	select {
 | 
			
		||||
	case <-w.result:
 | 
			
		||||
		t.Error("No result should be sent when already done")
 | 
			
		||||
	case <-time.After(time.Millisecond):
 | 
			
		||||
		// Expected
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestWantConn_cancel_NotDone(t *testing.T) {
 | 
			
		||||
	w := &wantConn{
 | 
			
		||||
		ctx:    context.Background(),
 | 
			
		||||
		result: make(chan wantConnResult, 1),
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Test cancel when not done
 | 
			
		||||
	cn := w.cancel()
 | 
			
		||||
 | 
			
		||||
	// Should return nil since no connection was not delivered
 | 
			
		||||
	if cn != nil {
 | 
			
		||||
		t.Errorf("cancel()= %v, want nil when no connection delivered", cn)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check that wantConn is marked as done
 | 
			
		||||
	if !w.done {
 | 
			
		||||
		t.Error("wantConn.done = false, want true after cancel")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check that context is cleared
 | 
			
		||||
	if w.ctx != nil {
 | 
			
		||||
		t.Error("wantConn.ctx should be nil after cancel")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check that channel is closed
 | 
			
		||||
	select {
 | 
			
		||||
	case _, ok := <-w.result:
 | 
			
		||||
		if ok {
 | 
			
		||||
			t.Error("result channel should be closed after cancel")
 | 
			
		||||
		}
 | 
			
		||||
	case <-time.After(time.Millisecond):
 | 
			
		||||
		t.Error("Expected channel to be closed")
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestWantConn_cancel_AlreadyDone(t *testing.T) {
 | 
			
		||||
	w := &wantConn{
 | 
			
		||||
		ctx:    context.Background(),
 | 
			
		||||
		done:   true,
 | 
			
		||||
		result: make(chan wantConnResult, 1),
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Put a result in the channel without connection (to avoid nil pointer issues)
 | 
			
		||||
	testErr := errors.New("test error")
 | 
			
		||||
	w.result <- wantConnResult{cn: nil, err: testErr}
 | 
			
		||||
 | 
			
		||||
	// Test cancel when already done
 | 
			
		||||
	cn := w.cancel()
 | 
			
		||||
 | 
			
		||||
	// Should return nil since the result had no connection
 | 
			
		||||
	if cn != nil {
 | 
			
		||||
		t.Errorf("cancel()= %v, want nil when result had no connection", cn)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check that wantConn remains done
 | 
			
		||||
	if !w.done {
 | 
			
		||||
		t.Error("wantConn.done = false, want true")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Check that context is cleared
 | 
			
		||||
	if w.ctx != nil {
 | 
			
		||||
		t.Error("wantConn.ctx should be nil after cancel")
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestWantConnQueue_newWantConnQueue(t *testing.T) {
 | 
			
		||||
	q := newWantConnQueue()
 | 
			
		||||
	if q == nil {
 | 
			
		||||
		t.Fatal("newWantConnQueue() returned nil")
 | 
			
		||||
	}
 | 
			
		||||
	if q.items == nil {
 | 
			
		||||
		t.Error("queue items should be initialized")
 | 
			
		||||
	}
 | 
			
		||||
	if len(q.items) != 0 {
 | 
			
		||||
		t.Errorf("new queue length = %d, want 0", len(q.items))
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestWantConnQueue_enqueue_dequeue(t *testing.T) {
 | 
			
		||||
	q := newWantConnQueue()
 | 
			
		||||
 | 
			
		||||
	// Test dequeue from empty queue
 | 
			
		||||
	item, ok := q.dequeue()
 | 
			
		||||
	if ok {
 | 
			
		||||
		t.Error("dequeue() from empty queue should return false")
 | 
			
		||||
	}
 | 
			
		||||
	if item != nil {
 | 
			
		||||
		t.Error("dequeue() from empty queue should return nil")
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Create test wantConn items
 | 
			
		||||
	w1 := &wantConn{ctx: context.Background(), result: make(chan wantConnResult, 1)}
 | 
			
		||||
	w2 := &wantConn{ctx: context.Background(), result: make(chan wantConnResult, 1)}
 | 
			
		||||
	w3 := &wantConn{ctx: context.Background(), result: make(chan wantConnResult, 1)}
 | 
			
		||||
 | 
			
		||||
	// Test enqueue
 | 
			
		||||
	q.enqueue(w1)
 | 
			
		||||
	q.enqueue(w2)
 | 
			
		||||
	q.enqueue(w3)
 | 
			
		||||
 | 
			
		||||
	// Test FIFO behavior
 | 
			
		||||
	item, ok = q.dequeue()
 | 
			
		||||
	if !ok {
 | 
			
		||||
		t.Error("dequeue() should return true when queue has items")
 | 
			
		||||
	}
 | 
			
		||||
	if item != w1 {
 | 
			
		||||
		t.Errorf("dequeue() = %v, want %v (FIFO order)", item, w1)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	item, ok = q.dequeue()
 | 
			
		||||
	if !ok {
 | 
			
		||||
		t.Error("dequeue() should return true when queue has items")
 | 
			
		||||
	}
 | 
			
		||||
	if item != w2 {
 | 
			
		||||
		t.Errorf("dequeue() = %v, want %v (FIFO order)", item, w2)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	item, ok = q.dequeue()
 | 
			
		||||
	if !ok {
 | 
			
		||||
		t.Error("dequeue() should return true when queue has items")
 | 
			
		||||
	}
 | 
			
		||||
	if item != w3 {
 | 
			
		||||
		t.Errorf("dequeue() = %v, want %v (FIFO order)", item, w3)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Test dequeue from empty queue again
 | 
			
		||||
	item, ok = q.dequeue()
 | 
			
		||||
	if ok {
 | 
			
		||||
		t.Error("dequeue() from empty queue should return false")
 | 
			
		||||
	}
 | 
			
		||||
	if item != nil {
 | 
			
		||||
		t.Error("dequeue() from empty queue should return nil")
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestWantConnQueue_ConcurrentAccess(t *testing.T) {
 | 
			
		||||
	q := newWantConnQueue()
 | 
			
		||||
	const numWorkers = 10
 | 
			
		||||
	const itemsPerWorker = 100
 | 
			
		||||
 | 
			
		||||
	var wg sync.WaitGroup
 | 
			
		||||
 | 
			
		||||
	// Start enqueuers
 | 
			
		||||
	for i := 0; i < numWorkers; i++ {
 | 
			
		||||
		wg.Add(1)
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer wg.Done()
 | 
			
		||||
			for j := 0; j < itemsPerWorker; j++ {
 | 
			
		||||
				w := &wantConn{
 | 
			
		||||
					ctx:    context.Background(),
 | 
			
		||||
					result: make(chan wantConnResult, 1),
 | 
			
		||||
				}
 | 
			
		||||
				q.enqueue(w)
 | 
			
		||||
			}
 | 
			
		||||
		}()
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Start dequeuers
 | 
			
		||||
	dequeued := make(chan *wantConn, numWorkers*itemsPerWorker)
 | 
			
		||||
	for i := 0; i < numWorkers; i++ {
 | 
			
		||||
		wg.Add(1)
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer wg.Done()
 | 
			
		||||
			for j := 0; j < itemsPerWorker; j++ {
 | 
			
		||||
				for {
 | 
			
		||||
					if item, ok := q.dequeue(); ok {
 | 
			
		||||
						dequeued <- item
 | 
			
		||||
						break
 | 
			
		||||
					}
 | 
			
		||||
					// Small delay to avoid busy waiting
 | 
			
		||||
					time.Sleep(time.Microsecond)
 | 
			
		||||
				}
 | 
			
		||||
			}
 | 
			
		||||
		}()
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	wg.Wait()
 | 
			
		||||
	close(dequeued)
 | 
			
		||||
 | 
			
		||||
	// Count dequeued items
 | 
			
		||||
	count := 0
 | 
			
		||||
	for range dequeued {
 | 
			
		||||
		count++
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	expectedCount := numWorkers * itemsPerWorker
 | 
			
		||||
	if count != expectedCount {
 | 
			
		||||
		t.Errorf("dequeued %d items, want %d", count, expectedCount)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Queue should be empty
 | 
			
		||||
	if item, ok := q.dequeue(); ok {
 | 
			
		||||
		t.Errorf("queue should be empty but got item: %v", item)
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestWantConnQueue_ThreadSafety(t *testing.T) {
 | 
			
		||||
	q := newWantConnQueue()
 | 
			
		||||
	const numOperations = 1000
 | 
			
		||||
 | 
			
		||||
	var wg sync.WaitGroup
 | 
			
		||||
	errors := make(chan error, numOperations*2)
 | 
			
		||||
 | 
			
		||||
	// Concurrent enqueue operations
 | 
			
		||||
	wg.Add(1)
 | 
			
		||||
	go func() {
 | 
			
		||||
		defer wg.Done()
 | 
			
		||||
		for i := 0; i < numOperations; i++ {
 | 
			
		||||
			w := &wantConn{
 | 
			
		||||
				ctx:    context.Background(),
 | 
			
		||||
				result: make(chan wantConnResult, 1),
 | 
			
		||||
			}
 | 
			
		||||
			q.enqueue(w)
 | 
			
		||||
		}
 | 
			
		||||
	}()
 | 
			
		||||
 | 
			
		||||
	// Concurrent dequeue operations
 | 
			
		||||
	wg.Add(1)
 | 
			
		||||
	go func() {
 | 
			
		||||
		defer wg.Done()
 | 
			
		||||
		dequeued := 0
 | 
			
		||||
		for dequeued < numOperations {
 | 
			
		||||
			if _, ok := q.dequeue(); ok {
 | 
			
		||||
				dequeued++
 | 
			
		||||
			} else {
 | 
			
		||||
				// Small delay when queue is empty
 | 
			
		||||
				time.Sleep(time.Microsecond)
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
	}()
 | 
			
		||||
 | 
			
		||||
	// Wait for completion
 | 
			
		||||
	wg.Wait()
 | 
			
		||||
	close(errors)
 | 
			
		||||
 | 
			
		||||
	// Check for any errors
 | 
			
		||||
	for err := range errors {
 | 
			
		||||
		t.Error(err)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Final queue should be empty
 | 
			
		||||
	if item, ok := q.dequeue(); ok {
 | 
			
		||||
		t.Errorf("queue should be empty but got item: %v", item)
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Benchmark tests
 | 
			
		||||
func BenchmarkWantConnQueue_Enqueue(b *testing.B) {
 | 
			
		||||
	q := newWantConnQueue()
 | 
			
		||||
 | 
			
		||||
	// Pre-allocate a pool of wantConn to reuse
 | 
			
		||||
	const poolSize = 1000
 | 
			
		||||
	wantConnPool := make([]*wantConn, poolSize)
 | 
			
		||||
	for i := 0; i < poolSize; i++ {
 | 
			
		||||
		wantConnPool[i] = &wantConn{
 | 
			
		||||
			ctx:    context.Background(),
 | 
			
		||||
			result: make(chan wantConnResult, 1),
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	b.ResetTimer()
 | 
			
		||||
 | 
			
		||||
	for i := 0; i < b.N; i++ {
 | 
			
		||||
		w := wantConnPool[i%poolSize]
 | 
			
		||||
		q.enqueue(w)
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func BenchmarkWantConnQueue_Dequeue(b *testing.B) {
 | 
			
		||||
	q := newWantConnQueue()
 | 
			
		||||
 | 
			
		||||
	// Use a reasonable fixed size for pre-population to avoid memory issues
 | 
			
		||||
	const queueSize = 10000
 | 
			
		||||
 | 
			
		||||
	// Pre-populate queue with a fixed reasonable size
 | 
			
		||||
	for i := 0; i < queueSize; i++ {
 | 
			
		||||
		w := &wantConn{
 | 
			
		||||
			ctx:    context.Background(),
 | 
			
		||||
			result: make(chan wantConnResult, 1),
 | 
			
		||||
		}
 | 
			
		||||
		q.enqueue(w)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	b.ResetTimer()
 | 
			
		||||
 | 
			
		||||
	// Benchmark dequeue operations, refilling as needed
 | 
			
		||||
	for i := 0; i < b.N; i++ {
 | 
			
		||||
		if _, ok := q.dequeue(); !ok {
 | 
			
		||||
			// Queue is empty, refill a batch
 | 
			
		||||
			for j := 0; j < 1000; j++ {
 | 
			
		||||
				w := &wantConn{
 | 
			
		||||
					ctx:    context.Background(),
 | 
			
		||||
					result: make(chan wantConnResult, 1),
 | 
			
		||||
				}
 | 
			
		||||
				q.enqueue(w)
 | 
			
		||||
			}
 | 
			
		||||
			// Dequeue again
 | 
			
		||||
			q.dequeue()
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func BenchmarkWantConnQueue_EnqueueDequeue(b *testing.B) {
 | 
			
		||||
	q := newWantConnQueue()
 | 
			
		||||
 | 
			
		||||
	// Pre-allocate a pool of wantConn to reuse
 | 
			
		||||
	const poolSize = 1000
 | 
			
		||||
	wantConnPool := make([]*wantConn, poolSize)
 | 
			
		||||
	for i := 0; i < poolSize; i++ {
 | 
			
		||||
		wantConnPool[i] = &wantConn{
 | 
			
		||||
			ctx:    context.Background(),
 | 
			
		||||
			result: make(chan wantConnResult, 1),
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	b.ResetTimer()
 | 
			
		||||
 | 
			
		||||
	for i := 0; i < b.N; i++ {
 | 
			
		||||
		w := wantConnPool[i%poolSize]
 | 
			
		||||
		q.enqueue(w)
 | 
			
		||||
		q.dequeue()
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
							
								
								
									
										13
									
								
								options.go
									
									
									
									
									
								
							
							
						
						
									
										13
									
								
								options.go
									
									
									
									
									
								
							@@ -34,7 +34,6 @@ type Limiter interface {
 | 
			
		||||
 | 
			
		||||
// Options keeps the settings to set up redis connection.
 | 
			
		||||
type Options struct {
 | 
			
		||||
 | 
			
		||||
	// Network type, either tcp or unix.
 | 
			
		||||
	//
 | 
			
		||||
	// default: is tcp.
 | 
			
		||||
@@ -176,6 +175,10 @@ type Options struct {
 | 
			
		||||
	// default: 10 * runtime.GOMAXPROCS(0)
 | 
			
		||||
	PoolSize int
 | 
			
		||||
 | 
			
		||||
	// MaxConcurrentDials is the maximum number of concurrent connection creation goroutines.
 | 
			
		||||
	// If <= 0, defaults to PoolSize. If > PoolSize, it will be capped at PoolSize.
 | 
			
		||||
	MaxConcurrentDials int
 | 
			
		||||
 | 
			
		||||
	// PoolTimeout is the amount of time client waits for connection if all connections
 | 
			
		||||
	// are busy before returning an error.
 | 
			
		||||
	//
 | 
			
		||||
@@ -295,6 +298,11 @@ func (opt *Options) init() {
 | 
			
		||||
	if opt.PoolSize == 0 {
 | 
			
		||||
		opt.PoolSize = 10 * runtime.GOMAXPROCS(0)
 | 
			
		||||
	}
 | 
			
		||||
	if opt.MaxConcurrentDials <= 0 {
 | 
			
		||||
		opt.MaxConcurrentDials = opt.PoolSize
 | 
			
		||||
	} else if opt.MaxConcurrentDials > opt.PoolSize {
 | 
			
		||||
		opt.MaxConcurrentDials = opt.PoolSize
 | 
			
		||||
	}
 | 
			
		||||
	if opt.ReadBufferSize == 0 {
 | 
			
		||||
		opt.ReadBufferSize = proto.DefaultBufferSize
 | 
			
		||||
	}
 | 
			
		||||
@@ -622,6 +630,7 @@ func setupConnParams(u *url.URL, o *Options) (*Options, error) {
 | 
			
		||||
	o.MinIdleConns = q.int("min_idle_conns")
 | 
			
		||||
	o.MaxIdleConns = q.int("max_idle_conns")
 | 
			
		||||
	o.MaxActiveConns = q.int("max_active_conns")
 | 
			
		||||
	o.MaxConcurrentDials = q.int("max_concurrent_dials")
 | 
			
		||||
	if q.has("conn_max_idle_time") {
 | 
			
		||||
		o.ConnMaxIdleTime = q.duration("conn_max_idle_time")
 | 
			
		||||
	} else {
 | 
			
		||||
@@ -688,6 +697,7 @@ func newConnPool(
 | 
			
		||||
		},
 | 
			
		||||
		PoolFIFO:                 opt.PoolFIFO,
 | 
			
		||||
		PoolSize:                 poolSize,
 | 
			
		||||
		MaxConcurrentDials:       opt.MaxConcurrentDials,
 | 
			
		||||
		PoolTimeout:              opt.PoolTimeout,
 | 
			
		||||
		DialTimeout:              opt.DialTimeout,
 | 
			
		||||
		DialerRetries:            opt.DialerRetries,
 | 
			
		||||
@@ -728,6 +738,7 @@ func newPubSubPool(opt *Options, dialer func(ctx context.Context, network, addr
 | 
			
		||||
	return pool.NewPubSubPool(&pool.Options{
 | 
			
		||||
		PoolFIFO:                 opt.PoolFIFO,
 | 
			
		||||
		PoolSize:                 poolSize,
 | 
			
		||||
		MaxConcurrentDials:       opt.MaxConcurrentDials,
 | 
			
		||||
		PoolTimeout:              opt.PoolTimeout,
 | 
			
		||||
		DialTimeout:              opt.DialTimeout,
 | 
			
		||||
		DialerRetries:            opt.DialerRetries,
 | 
			
		||||
 
 | 
			
		||||
@@ -67,6 +67,12 @@ func TestParseURL(t *testing.T) {
 | 
			
		||||
		}, {
 | 
			
		||||
			url: "redis://localhost:123/?db=2&protocol=2", // RESP Protocol
 | 
			
		||||
			o:   &Options{Addr: "localhost:123", DB: 2, Protocol: 2},
 | 
			
		||||
		}, {
 | 
			
		||||
			url: "redis://localhost:123/?max_concurrent_dials=5", // MaxConcurrentDials parameter
 | 
			
		||||
			o:   &Options{Addr: "localhost:123", MaxConcurrentDials: 5},
 | 
			
		||||
		}, {
 | 
			
		||||
			url: "redis://localhost:123/?max_concurrent_dials=0", // MaxConcurrentDials zero value
 | 
			
		||||
			o:   &Options{Addr: "localhost:123", MaxConcurrentDials: 0},
 | 
			
		||||
		}, {
 | 
			
		||||
			url: "unix:///tmp/redis.sock",
 | 
			
		||||
			o:   &Options{Addr: "/tmp/redis.sock"},
 | 
			
		||||
@@ -197,6 +203,9 @@ func comprareOptions(t *testing.T, actual, expected *Options) {
 | 
			
		||||
	if actual.ConnMaxLifetime != expected.ConnMaxLifetime {
 | 
			
		||||
		t.Errorf("ConnMaxLifetime: got %v, expected %v", actual.ConnMaxLifetime, expected.ConnMaxLifetime)
 | 
			
		||||
	}
 | 
			
		||||
	if actual.MaxConcurrentDials != expected.MaxConcurrentDials {
 | 
			
		||||
		t.Errorf("MaxConcurrentDials: got %v, expected %v", actual.MaxConcurrentDials, expected.MaxConcurrentDials)
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// Test ReadTimeout option initialization, including special values -1 and 0.
 | 
			
		||||
@@ -245,3 +254,68 @@ func TestProtocolOptions(t *testing.T) {
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestMaxConcurrentDialsOptions(t *testing.T) {
 | 
			
		||||
	// Test cases for MaxConcurrentDials initialization logic
 | 
			
		||||
	testCases := []struct {
 | 
			
		||||
		name                    string
 | 
			
		||||
		poolSize                int
 | 
			
		||||
		maxConcurrentDials      int
 | 
			
		||||
		expectedConcurrentDials int
 | 
			
		||||
	}{
 | 
			
		||||
		// Edge cases and invalid values - negative/zero values set to PoolSize
 | 
			
		||||
		{
 | 
			
		||||
			name:                    "negative value gets set to pool size",
 | 
			
		||||
			poolSize:                10,
 | 
			
		||||
			maxConcurrentDials:      -1,
 | 
			
		||||
			expectedConcurrentDials: 10, // negative values are set to PoolSize
 | 
			
		||||
		},
 | 
			
		||||
		// Zero value tests - MaxConcurrentDials should be set to PoolSize
 | 
			
		||||
		{
 | 
			
		||||
			name:                    "zero value with positive pool size",
 | 
			
		||||
			poolSize:                1,
 | 
			
		||||
			maxConcurrentDials:      0,
 | 
			
		||||
			expectedConcurrentDials: 1, // MaxConcurrentDials = PoolSize when 0
 | 
			
		||||
		},
 | 
			
		||||
		// Explicit positive value tests
 | 
			
		||||
		{
 | 
			
		||||
			name:                    "explicit value within limit",
 | 
			
		||||
			poolSize:                10,
 | 
			
		||||
			maxConcurrentDials:      3,
 | 
			
		||||
			expectedConcurrentDials: 3, // should remain unchanged when < PoolSize
 | 
			
		||||
		},
 | 
			
		||||
		// Capping tests - values exceeding PoolSize should be capped
 | 
			
		||||
		{
 | 
			
		||||
			name:                    "value exceeding pool size",
 | 
			
		||||
			poolSize:                5,
 | 
			
		||||
			maxConcurrentDials:      10,
 | 
			
		||||
			expectedConcurrentDials: 5, // should be capped at PoolSize
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	for _, tc := range testCases {
 | 
			
		||||
		t.Run(tc.name, func(t *testing.T) {
 | 
			
		||||
			opts := &Options{
 | 
			
		||||
				PoolSize:           tc.poolSize,
 | 
			
		||||
				MaxConcurrentDials: tc.maxConcurrentDials,
 | 
			
		||||
			}
 | 
			
		||||
			opts.init()
 | 
			
		||||
 | 
			
		||||
			if opts.MaxConcurrentDials != tc.expectedConcurrentDials {
 | 
			
		||||
				t.Errorf("MaxConcurrentDials: got %v, expected %v (PoolSize=%v)",
 | 
			
		||||
					opts.MaxConcurrentDials, tc.expectedConcurrentDials, opts.PoolSize)
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			// Ensure MaxConcurrentDials never exceeds PoolSize (for all inputs)
 | 
			
		||||
			if opts.MaxConcurrentDials > opts.PoolSize {
 | 
			
		||||
				t.Errorf("MaxConcurrentDials (%v) should not exceed PoolSize (%v)",
 | 
			
		||||
					opts.MaxConcurrentDials, opts.PoolSize)
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			// Ensure MaxConcurrentDials is always positive (for all inputs)
 | 
			
		||||
			if opts.MaxConcurrentDials <= 0 {
 | 
			
		||||
				t.Errorf("MaxConcurrentDials should be positive, got %v", opts.MaxConcurrentDials)
 | 
			
		||||
			}
 | 
			
		||||
		})
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 
 | 
			
		||||
@@ -72,6 +72,7 @@ func BenchmarkPoolGetPut(b *testing.B) {
 | 
			
		||||
			connPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
				Dialer:             dummyDialer,
 | 
			
		||||
				PoolSize:           int32(poolSize),
 | 
			
		||||
				MaxConcurrentDials: poolSize,
 | 
			
		||||
				PoolTimeout:        time.Second,
 | 
			
		||||
				DialTimeout:        time.Second,
 | 
			
		||||
				ConnMaxIdleTime:    time.Hour,
 | 
			
		||||
@@ -114,6 +115,7 @@ func BenchmarkPoolGetPutWithMinIdle(b *testing.B) {
 | 
			
		||||
			connPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
				Dialer:             dummyDialer,
 | 
			
		||||
				PoolSize:           int32(config.poolSize),
 | 
			
		||||
				MaxConcurrentDials: config.poolSize,
 | 
			
		||||
				MinIdleConns:       int32(config.minIdleConns),
 | 
			
		||||
				PoolTimeout:        time.Second,
 | 
			
		||||
				DialTimeout:        time.Second,
 | 
			
		||||
@@ -144,6 +146,7 @@ func BenchmarkPoolConcurrentGetPut(b *testing.B) {
 | 
			
		||||
	connPool := pool.NewConnPool(&pool.Options{
 | 
			
		||||
		Dialer:             dummyDialer,
 | 
			
		||||
		PoolSize:           int32(32),
 | 
			
		||||
		MaxConcurrentDials: 32,
 | 
			
		||||
		PoolTimeout:        time.Second,
 | 
			
		||||
		DialTimeout:        time.Second,
 | 
			
		||||
		ConnMaxIdleTime:    time.Hour,
 | 
			
		||||
 
 | 
			
		||||
		Reference in New Issue
	
	Block a user