1
0
mirror of https://github.com/redis/go-redis.git synced 2025-11-02 15:33:16 +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:
cyningsun
2025-10-31 01:21:12 +08:00
committed by GitHub
parent 7f48276660
commit ae5434ce66
11 changed files with 1361 additions and 97 deletions

View File

@@ -53,8 +53,9 @@ func TestEventDrivenHandoffIntegration(t *testing.T) {
Dialer: func(ctx context.Context) (net.Conn, error) {
return &mockNetConn{addr: "original:6379"}, nil
},
PoolSize: int32(5),
PoolTimeout: time.Second,
PoolSize: int32(5),
MaxConcurrentDials: 5,
PoolTimeout: time.Second,
})
// Add the hook to the pool after creation
@@ -153,8 +154,9 @@ func TestEventDrivenHandoffIntegration(t *testing.T) {
return &mockNetConn{addr: "original:6379"}, nil
},
PoolSize: int32(10),
PoolTimeout: time.Second,
PoolSize: int32(10),
MaxConcurrentDials: 10,
PoolTimeout: time.Second,
})
defer testPool.Close()
@@ -225,8 +227,9 @@ func TestEventDrivenHandoffIntegration(t *testing.T) {
return &mockNetConn{addr: "original:6379"}, nil
},
PoolSize: int32(3),
PoolTimeout: time.Second,
PoolSize: int32(3),
MaxConcurrentDials: 3,
PoolTimeout: time.Second,
})
defer testPool.Close()
@@ -288,8 +291,9 @@ func TestEventDrivenHandoffIntegration(t *testing.T) {
return &mockNetConn{addr: "original:6379"}, nil
},
PoolSize: int32(2),
PoolTimeout: time.Second,
PoolSize: int32(2),
MaxConcurrentDials: 2,
PoolTimeout: time.Second,
})
defer testPool.Close()

View File

@@ -31,11 +31,12 @@ func BenchmarkPoolGetPut(b *testing.B) {
for _, bm := range benchmarks {
b.Run(bm.String(), func(b *testing.B) {
connPool := pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(bm.poolSize),
PoolTimeout: time.Second,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Hour,
Dialer: dummyDialer,
PoolSize: int32(bm.poolSize),
MaxConcurrentDials: bm.poolSize,
PoolTimeout: time.Second,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Hour,
})
b.ResetTimer()
@@ -75,11 +76,12 @@ func BenchmarkPoolGetRemove(b *testing.B) {
for _, bm := range benchmarks {
b.Run(bm.String(), func(b *testing.B) {
connPool := pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(bm.poolSize),
PoolTimeout: time.Second,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Hour,
Dialer: dummyDialer,
PoolSize: int32(bm.poolSize),
MaxConcurrentDials: bm.poolSize,
PoolTimeout: time.Second,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Hour,
})
b.ResetTimer()

View File

@@ -24,9 +24,10 @@ var _ = Describe("Buffer Size Configuration", func() {
It("should use default buffer sizes when not specified", func() {
connPool = pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(1),
PoolTimeout: 1000,
Dialer: dummyDialer,
PoolSize: int32(1),
MaxConcurrentDials: 1,
PoolTimeout: 1000,
})
cn, err := connPool.NewConn(ctx)
@@ -46,11 +47,12 @@ var _ = Describe("Buffer Size Configuration", func() {
customWriteSize := 64 * 1024 // 64KB
connPool = pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(1),
PoolTimeout: 1000,
ReadBufferSize: customReadSize,
WriteBufferSize: customWriteSize,
Dialer: dummyDialer,
PoolSize: int32(1),
MaxConcurrentDials: 1,
PoolTimeout: 1000,
ReadBufferSize: customReadSize,
WriteBufferSize: customWriteSize,
})
cn, err := connPool.NewConn(ctx)
@@ -67,11 +69,12 @@ var _ = Describe("Buffer Size Configuration", func() {
It("should handle zero buffer sizes by using defaults", func() {
connPool = pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(1),
PoolTimeout: 1000,
ReadBufferSize: 0, // Should use default
WriteBufferSize: 0, // Should use default
Dialer: dummyDialer,
PoolSize: int32(1),
MaxConcurrentDials: 1,
PoolTimeout: 1000,
ReadBufferSize: 0, // Should use default
WriteBufferSize: 0, // Should use default
})
cn, err := connPool.NewConn(ctx)
@@ -103,9 +106,10 @@ var _ = Describe("Buffer Size Configuration", func() {
// Test the scenario where someone creates a pool directly (like in tests)
// without setting ReadBufferSize and WriteBufferSize
connPool = pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(1),
PoolTimeout: 1000,
Dialer: dummyDialer,
PoolSize: int32(1),
MaxConcurrentDials: 1,
PoolTimeout: 1000,
// ReadBufferSize and WriteBufferSize are not set (will be 0)
})

View File

@@ -191,8 +191,9 @@ func TestPoolWithHooks(t *testing.T) {
Dialer: func(ctx context.Context) (net.Conn, error) {
return &net.TCPConn{}, nil // Mock connection
},
PoolSize: 1,
DialTimeout: time.Second,
PoolSize: 1,
MaxConcurrentDials: 1,
DialTimeout: time.Second,
}
pool := NewConnPool(opt)

View File

@@ -98,6 +98,7 @@ type Options struct {
PoolFIFO bool
PoolSize int32
MaxConcurrentDials int
DialTimeout time.Duration
PoolTimeout time.Duration
MinIdleConns int32
@@ -126,7 +127,9 @@ type ConnPool struct {
dialErrorsNum uint32 // atomic
lastDialError atomic.Value
queue chan struct{}
queue chan struct{}
dialsInProgress chan struct{}
dialsQueue *wantConnQueue
connsMu sync.Mutex
conns map[uint64]*Conn
@@ -152,9 +155,11 @@ func NewConnPool(opt *Options) *ConnPool {
p := &ConnPool{
cfg: opt,
queue: make(chan struct{}, opt.PoolSize),
conns: make(map[uint64]*Conn),
idleConns: make([]*Conn, 0, opt.PoolSize),
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),
}
// Only create MinIdleConns if explicitly requested (> 0)
@@ -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():

View File

@@ -3,6 +3,7 @@ package pool_test
import (
"context"
"errors"
"fmt"
"net"
"sync"
"sync/atomic"
@@ -21,11 +22,12 @@ var _ = Describe("ConnPool", func() {
BeforeEach(func() {
connPool = pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(10),
PoolTimeout: time.Hour,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Millisecond,
Dialer: dummyDialer,
PoolSize: int32(10),
MaxConcurrentDials: 10,
PoolTimeout: time.Hour,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Millisecond,
})
})
@@ -47,17 +49,18 @@ var _ = Describe("ConnPool", func() {
<-closedChan
return &net.TCPConn{}, nil
},
PoolSize: int32(10),
PoolTimeout: time.Hour,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Millisecond,
MinIdleConns: int32(minIdleConns),
PoolSize: int32(10),
MaxConcurrentDials: 10,
PoolTimeout: time.Hour,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Millisecond,
MinIdleConns: int32(minIdleConns),
})
wg.Wait()
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{
@@ -131,12 +134,13 @@ var _ = Describe("MinIdleConns", func() {
newConnPool := func() *pool.ConnPool {
connPool := pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(poolSize),
MinIdleConns: int32(minIdleConns),
PoolTimeout: 100 * time.Millisecond,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: -1,
Dialer: dummyDialer,
PoolSize: int32(poolSize),
MaxConcurrentDials: poolSize,
MinIdleConns: int32(minIdleConns),
PoolTimeout: 100 * time.Millisecond,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: -1,
})
Eventually(func() int {
return connPool.Len()
@@ -310,11 +314,12 @@ var _ = Describe("race", func() {
It("does not happen on Get, Put, and Remove", func() {
connPool = pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(10),
PoolTimeout: time.Minute,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Millisecond,
Dialer: dummyDialer,
PoolSize: int32(10),
MaxConcurrentDials: 10,
PoolTimeout: time.Minute,
DialTimeout: 1 * time.Second,
ConnMaxIdleTime: time.Millisecond,
})
perform(C, func(id int) {
@@ -341,10 +346,11 @@ var _ = Describe("race", func() {
Dialer: func(ctx context.Context) (net.Conn, error) {
return &net.TCPConn{}, nil
},
PoolSize: int32(1000),
MinIdleConns: int32(50),
PoolTimeout: 3 * time.Second,
DialTimeout: 1 * time.Second,
PoolSize: int32(1000),
MaxConcurrentDials: 1000,
MinIdleConns: int32(50),
PoolTimeout: 3 * time.Second,
DialTimeout: 1 * time.Second,
}
p := pool.NewConnPool(opt)
@@ -368,8 +374,9 @@ var _ = Describe("race", func() {
Dialer: func(ctx context.Context) (net.Conn, error) {
panic("test panic")
},
PoolSize: int32(100),
MinIdleConns: int32(30),
PoolSize: int32(100),
MaxConcurrentDials: 100,
MinIdleConns: int32(30),
}
p := pool.NewConnPool(opt)
@@ -386,8 +393,9 @@ var _ = Describe("race", func() {
Dialer: func(ctx context.Context) (net.Conn, error) {
return &net.TCPConn{}, nil
},
PoolSize: int32(1),
PoolTimeout: 3 * time.Second,
PoolSize: int32(1),
MaxConcurrentDials: 1,
PoolTimeout: 3 * time.Second,
}
p := pool.NewConnPool(opt)
@@ -417,8 +425,9 @@ var _ = Describe("race", func() {
return &net.TCPConn{}, nil
},
PoolSize: int32(1),
PoolTimeout: testPoolTimeout,
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
@@ -483,10 +493,11 @@ func TestDialerRetryConfiguration(t *testing.T) {
}
connPool := pool.NewConnPool(&pool.Options{
Dialer: failingDialer,
PoolSize: 1,
PoolTimeout: time.Second,
DialTimeout: time.Second,
Dialer: failingDialer,
PoolSize: 1,
MaxConcurrentDials: 1,
PoolTimeout: time.Second,
DialTimeout: time.Second,
// DialerRetries and DialerRetryTimeout not set - should use defaults
})
defer connPool.Close()
@@ -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()
}

View 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
}

View 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()
}
}

View File

@@ -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,

View File

@@ -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)
}
})
}
}

View File

@@ -70,12 +70,13 @@ func BenchmarkPoolGetPut(b *testing.B) {
for _, poolSize := range poolSizes {
b.Run(fmt.Sprintf("PoolSize_%d", poolSize), func(b *testing.B) {
connPool := pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(poolSize),
PoolTimeout: time.Second,
DialTimeout: time.Second,
ConnMaxIdleTime: time.Hour,
MinIdleConns: int32(0), // Start with no idle connections
Dialer: dummyDialer,
PoolSize: int32(poolSize),
MaxConcurrentDials: poolSize,
PoolTimeout: time.Second,
DialTimeout: time.Second,
ConnMaxIdleTime: time.Hour,
MinIdleConns: int32(0), // Start with no idle connections
})
defer connPool.Close()
@@ -112,12 +113,13 @@ func BenchmarkPoolGetPutWithMinIdle(b *testing.B) {
for _, config := range configs {
b.Run(fmt.Sprintf("Pool_%d_MinIdle_%d", config.poolSize, config.minIdleConns), func(b *testing.B) {
connPool := pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(config.poolSize),
MinIdleConns: int32(config.minIdleConns),
PoolTimeout: time.Second,
DialTimeout: time.Second,
ConnMaxIdleTime: time.Hour,
Dialer: dummyDialer,
PoolSize: int32(config.poolSize),
MaxConcurrentDials: config.poolSize,
MinIdleConns: int32(config.minIdleConns),
PoolTimeout: time.Second,
DialTimeout: time.Second,
ConnMaxIdleTime: time.Hour,
})
defer connPool.Close()
@@ -142,12 +144,13 @@ func BenchmarkPoolConcurrentGetPut(b *testing.B) {
ctx := context.Background()
connPool := pool.NewConnPool(&pool.Options{
Dialer: dummyDialer,
PoolSize: int32(32),
PoolTimeout: time.Second,
DialTimeout: time.Second,
ConnMaxIdleTime: time.Hour,
MinIdleConns: int32(0),
Dialer: dummyDialer,
PoolSize: int32(32),
MaxConcurrentDials: 32,
PoolTimeout: time.Second,
DialTimeout: time.Second,
ConnMaxIdleTime: time.Hour,
MinIdleConns: int32(0),
})
defer connPool.Close()