1
0
mirror of https://github.com/redis/go-redis.git synced 2025-07-29 17:41:15 +03:00

Faster and simpler pool.

This commit is contained in:
Vladimir Mihailenco
2016-03-17 18:00:47 +02:00
parent 93a7fe0de3
commit 6e1aef39ea
22 changed files with 418 additions and 492 deletions

View File

@ -2,7 +2,6 @@ package pool_test
import (
"errors"
"net"
"testing"
"time"
@ -10,22 +9,19 @@ import (
)
func benchmarkPoolGetPut(b *testing.B, poolSize int) {
dial := func() (net.Conn, error) {
return &net.TCPConn{}, nil
}
pool := pool.NewConnPool(dial, poolSize, time.Second, 0)
pool.DialLimiter = nil
connPool := pool.NewConnPool(dummyDialer, poolSize, time.Second, time.Hour, time.Hour)
connPool.DialLimiter = nil
b.ResetTimer()
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
conn, err := pool.Get()
cn, err := connPool.Get()
if err != nil {
b.Fatalf("no error expected on pool.Get but received: %s", err.Error())
b.Fatal(err)
}
if err = pool.Put(conn); err != nil {
b.Fatalf("no error expected on pool.Put but received: %s", err.Error())
if err = connPool.Put(cn); err != nil {
b.Fatal(err)
}
}
})
@ -43,38 +39,34 @@ func BenchmarkPoolGetPut1000Conns(b *testing.B) {
benchmarkPoolGetPut(b, 1000)
}
func benchmarkPoolGetReplace(b *testing.B, poolSize int) {
dial := func() (net.Conn, error) {
return &net.TCPConn{}, nil
}
pool := pool.NewConnPool(dial, poolSize, time.Second, 0)
pool.DialLimiter = nil
func benchmarkPoolGetRemove(b *testing.B, poolSize int) {
connPool := pool.NewConnPool(dummyDialer, poolSize, time.Second, time.Hour, time.Hour)
connPool.DialLimiter = nil
removeReason := errors.New("benchmark")
b.ResetTimer()
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
conn, err := pool.Get()
cn, err := connPool.Get()
if err != nil {
b.Fatalf("no error expected on pool.Get but received: %s", err.Error())
b.Fatal(err)
}
if err = pool.Replace(conn, removeReason); err != nil {
b.Fatalf("no error expected on pool.Remove but received: %s", err.Error())
if err := connPool.Remove(cn, removeReason); err != nil {
b.Fatal(err)
}
}
})
}
func BenchmarkPoolGetReplace10Conns(b *testing.B) {
benchmarkPoolGetReplace(b, 10)
func BenchmarkPoolGetRemove10Conns(b *testing.B) {
benchmarkPoolGetRemove(b, 10)
}
func BenchmarkPoolGetReplace100Conns(b *testing.B) {
benchmarkPoolGetReplace(b, 100)
func BenchmarkPoolGetRemove100Conns(b *testing.B) {
benchmarkPoolGetRemove(b, 100)
}
func BenchmarkPoolGetReplace1000Conns(b *testing.B) {
benchmarkPoolGetReplace(b, 1000)
func BenchmarkPoolGetRemove1000Conns(b *testing.B) {
benchmarkPoolGetRemove(b, 1000)
}

View File

@ -4,7 +4,6 @@ import (
"bufio"
"io"
"net"
"sync/atomic"
"time"
)
@ -13,8 +12,6 @@ const defaultBufSize = 4096
var noDeadline = time.Time{}
type Conn struct {
idx int32
NetConn net.Conn
Rd *bufio.Reader
Buf []byte
@ -28,8 +25,6 @@ type Conn struct {
func NewConn(netConn net.Conn) *Conn {
cn := &Conn{
idx: -1,
NetConn: netConn,
Buf: make([]byte, defaultBufSize),
@ -39,18 +34,6 @@ func NewConn(netConn net.Conn) *Conn {
return cn
}
func (cn *Conn) Index() int {
return int(atomic.LoadInt32(&cn.idx))
}
func (cn *Conn) SetIndex(newIdx int) int {
oldIdx := cn.Index()
if !atomic.CompareAndSwapInt32(&cn.idx, int32(oldIdx), int32(newIdx)) {
return -1
}
return oldIdx
}
func (cn *Conn) IsStale(timeout time.Duration) bool {
return timeout > 0 && time.Since(cn.UsedAt) > timeout
}

View File

@ -1,89 +0,0 @@
package pool
import (
"sync"
"sync/atomic"
)
type connList struct {
cns []*Conn
mu sync.Mutex
len int32 // atomic
size int32
}
func newConnList(size int) *connList {
return &connList{
cns: make([]*Conn, size),
size: int32(size),
}
}
func (l *connList) Len() int {
return int(atomic.LoadInt32(&l.len))
}
// Reserve reserves place in the list and returns true on success.
// The caller must add connection or cancel reservation if it was reserved.
func (l *connList) Reserve() bool {
len := atomic.AddInt32(&l.len, 1)
reserved := len <= l.size
if !reserved {
atomic.AddInt32(&l.len, -1)
}
return reserved
}
func (l *connList) CancelReservation() {
atomic.AddInt32(&l.len, -1)
}
// Add adds connection to the list. The caller must reserve place first.
func (l *connList) Add(cn *Conn) {
l.mu.Lock()
for i, c := range l.cns {
if c == nil {
cn.SetIndex(i)
l.cns[i] = cn
l.mu.Unlock()
return
}
}
panic("not reached")
}
func (l *connList) Replace(cn *Conn) {
l.mu.Lock()
if l.cns != nil {
l.cns[cn.idx] = cn
}
l.mu.Unlock()
}
// Remove closes connection and removes it from the list.
func (l *connList) Remove(idx int) {
l.mu.Lock()
if l.cns != nil {
l.cns[idx] = nil
atomic.AddInt32(&l.len, -1)
}
l.mu.Unlock()
}
func (l *connList) Reset() []*Conn {
l.mu.Lock()
for _, cn := range l.cns {
if cn == nil {
continue
}
cn.SetIndex(-1)
}
cns := l.cns
l.cns = nil
l.len = 0
l.mu.Unlock()
return cns
}

View File

@ -1,74 +0,0 @@
package pool
import (
"sync"
"time"
)
// connStack is used as a LIFO to maintain free connections
type connStack struct {
cns []*Conn
free chan struct{}
mu sync.Mutex
}
func newConnStack(max int) *connStack {
return &connStack{
cns: make([]*Conn, 0, max),
free: make(chan struct{}, max),
}
}
func (s *connStack) Len() int { return len(s.free) }
func (s *connStack) Push(cn *Conn) {
s.mu.Lock()
s.cns = append(s.cns, cn)
s.mu.Unlock()
s.free <- struct{}{}
}
func (s *connStack) ShiftStale(idleTimeout time.Duration) *Conn {
select {
case <-s.free:
s.mu.Lock()
if cn := s.cns[0]; cn.IsStale(idleTimeout) {
copy(s.cns, s.cns[1:])
s.cns = s.cns[:len(s.cns)-1]
s.mu.Unlock()
return cn
}
s.mu.Unlock()
s.free <- struct{}{}
return nil
default:
return nil
}
}
func (s *connStack) Pop() *Conn {
select {
case <-s.free:
return s.pop()
default:
return nil
}
}
func (s *connStack) PopWithTimeout(d time.Duration) *Conn {
select {
case <-s.free:
return s.pop()
case <-time.After(d):
return nil
}
}
func (s *connStack) pop() (cn *Conn) {
s.mu.Lock()
ci := len(s.cns) - 1
cn, s.cns = s.cns[ci], s.cns[:ci]
s.mu.Unlock()
return
}

View File

@ -3,24 +3,34 @@ package pool
import (
"errors"
"fmt"
"io/ioutil"
"log"
"net"
"os"
"sync"
"sync/atomic"
"time"
"gopkg.in/bsm/ratelimit.v1"
)
var Logger = log.New(os.Stderr, "redis: ", log.LstdFlags)
var Logger = log.New(ioutil.Discard, "redis: ", log.LstdFlags)
var (
ErrClosed = errors.New("redis: client is closed")
errConnClosed = errors.New("redis: connection is closed")
ErrPoolTimeout = errors.New("redis: connection pool timeout")
errConnClosed = errors.New("connection is closed")
errConnStale = errors.New("connection is stale")
)
var timers = sync.Pool{
New: func() interface{} {
return time.NewTimer(0)
},
}
// PoolStats contains pool state information and accumulated stats.
// TODO: remove Waits
type PoolStats struct {
Requests uint32 // number of times a connection was requested by the pool
Hits uint32 // number of times free connection was found in the pool
@ -32,10 +42,9 @@ type PoolStats struct {
}
type Pooler interface {
First() *Conn
Get() (*Conn, error)
Put(*Conn) error
Replace(*Conn, error) error
Remove(*Conn, error) error
Len() int
FreeLen() int
Stats() *PoolStats
@ -53,18 +62,23 @@ type ConnPool struct {
poolTimeout time.Duration
idleTimeout time.Duration
conns *connList
freeConns *connStack
stats PoolStats
queue chan struct{}
_closed int32
connsMu sync.Mutex
conns []*Conn
freeConnsMu sync.Mutex
freeConns []*Conn
stats PoolStats
_closed int32 // atomic
lastErr atomic.Value
}
var _ Pooler = (*ConnPool)(nil)
func NewConnPool(dial dialer, poolSize int, poolTimeout, idleTimeout time.Duration) *ConnPool {
func NewConnPool(dial dialer, poolSize int, poolTimeout, idleTimeout, idleCheckFrequency time.Duration) *ConnPool {
p := &ConnPool{
_dial: dial,
DialLimiter: ratelimit.New(3*poolSize, time.Second),
@ -72,57 +86,19 @@ func NewConnPool(dial dialer, poolSize int, poolTimeout, idleTimeout time.Durati
poolTimeout: poolTimeout,
idleTimeout: idleTimeout,
conns: newConnList(poolSize),
freeConns: newConnStack(poolSize),
queue: make(chan struct{}, poolSize),
conns: make([]*Conn, 0, poolSize),
freeConns: make([]*Conn, 0, poolSize),
}
if idleTimeout > 0 {
go p.reaper(getIdleCheckFrequency())
for i := 0; i < poolSize; i++ {
p.queue <- struct{}{}
}
if idleTimeout > 0 && idleCheckFrequency > 0 {
go p.reaper(idleCheckFrequency)
}
return p
}
func (p *ConnPool) Add(cn *Conn) bool {
if !p.conns.Reserve() {
return false
}
p.conns.Add(cn)
p.Put(cn)
return true
}
// First returns first non-idle connection from the pool or nil if
// there are no connections.
func (p *ConnPool) First() *Conn {
for {
cn := p.freeConns.Pop()
if cn != nil && cn.IsStale(p.idleTimeout) {
var err error
cn, err = p.replace(cn)
if err != nil {
Logger.Printf("pool.replace failed: %s", err)
continue
}
}
return cn
}
}
// wait waits for free non-idle connection. It returns nil on timeout.
func (p *ConnPool) wait(timeout time.Duration) *Conn {
for {
cn := p.freeConns.PopWithTimeout(timeout)
if cn != nil && cn.IsStale(p.idleTimeout) {
var err error
cn, err = p.replace(cn)
if err != nil {
Logger.Printf("pool.replace failed: %s", err)
continue
}
}
return cn
}
}
func (p *ConnPool) dial() (net.Conn, error) {
if p.DialLimiter != nil && p.DialLimiter.Limit() {
err := fmt.Errorf(
@ -148,6 +124,42 @@ func (p *ConnPool) NewConn() (*Conn, error) {
return NewConn(netConn), nil
}
func (p *ConnPool) PopFree() *Conn {
timer := timers.Get().(*time.Timer)
if !timer.Reset(p.poolTimeout) {
<-timer.C
}
select {
case <-p.queue:
timers.Put(timer)
case <-timer.C:
timers.Put(timer)
atomic.AddUint32(&p.stats.Timeouts, 1)
return nil
}
p.freeConnsMu.Lock()
cn := p.popFree()
p.freeConnsMu.Unlock()
if cn == nil {
p.queue <- struct{}{}
}
return cn
}
func (p *ConnPool) popFree() *Conn {
if len(p.freeConns) == 0 {
return nil
}
idx := len(p.freeConns) - 1
cn := p.freeConns[idx]
p.freeConns = p.freeConns[:idx]
return cn
}
// Get returns existed connection from the pool or creates a new one.
func (p *ConnPool) Get() (*Conn, error) {
if p.Closed() {
@ -156,31 +168,46 @@ func (p *ConnPool) Get() (*Conn, error) {
atomic.AddUint32(&p.stats.Requests, 1)
// Fetch first non-idle connection, if available.
if cn := p.First(); cn != nil {
timer := timers.Get().(*time.Timer)
if !timer.Reset(p.poolTimeout) {
<-timer.C
}
select {
case <-p.queue:
timers.Put(timer)
case <-timer.C:
timers.Put(timer)
atomic.AddUint32(&p.stats.Timeouts, 1)
return nil, ErrPoolTimeout
}
p.freeConnsMu.Lock()
cn := p.popFree()
p.freeConnsMu.Unlock()
if cn != nil {
atomic.AddUint32(&p.stats.Hits, 1)
return cn, nil
}
// Try to create a new one.
if p.conns.Reserve() {
cn, err := p.NewConn()
if err != nil {
p.conns.CancelReservation()
return nil, err
if !cn.IsStale(p.idleTimeout) {
return cn, nil
}
p.conns.Add(cn)
return cn, nil
_ = cn.Close()
}
// Otherwise, wait for the available connection.
atomic.AddUint32(&p.stats.Waits, 1)
if cn := p.wait(p.poolTimeout); cn != nil {
return cn, nil
newcn, err := p.NewConn()
if err != nil {
p.queue <- struct{}{}
return nil, err
}
atomic.AddUint32(&p.stats.Timeouts, 1)
return nil, ErrPoolTimeout
p.connsMu.Lock()
if cn != nil {
p.remove(cn, errConnStale)
}
p.conns = append(p.conns, newcn)
p.connsMu.Unlock()
return newcn, nil
}
func (p *ConnPool) Put(cn *Conn) error {
@ -188,71 +215,54 @@ func (p *ConnPool) Put(cn *Conn) error {
b, _ := cn.Rd.Peek(cn.Rd.Buffered())
err := fmt.Errorf("connection has unread data: %q", b)
Logger.Print(err)
return p.Replace(cn, err)
return p.Remove(cn, err)
}
p.freeConns.Push(cn)
return nil
}
func (p *ConnPool) replace(cn *Conn) (*Conn, error) {
_ = cn.Close()
idx := cn.SetIndex(-1)
if idx == -1 {
return nil, errConnClosed
}
netConn, err := p.dial()
if err != nil {
p.conns.Remove(idx)
return nil, err
}
cn = NewConn(netConn)
cn.SetIndex(idx)
p.conns.Replace(cn)
return cn, nil
}
func (p *ConnPool) Replace(cn *Conn, reason error) error {
p.storeLastErr(reason.Error())
// Replace existing connection with new one and unblock waiter.
newcn, err := p.replace(cn)
if err != nil {
return err
}
p.freeConns.Push(newcn)
p.freeConnsMu.Lock()
p.freeConns = append(p.freeConns, cn)
p.freeConnsMu.Unlock()
p.queue <- struct{}{}
return nil
}
func (p *ConnPool) Remove(cn *Conn, reason error) error {
_ = cn.Close()
idx := cn.SetIndex(-1)
if idx == -1 {
return errConnClosed
}
p.storeLastErr(reason.Error())
p.conns.Remove(idx)
p.connsMu.Lock()
p.remove(cn, reason)
p.connsMu.Unlock()
p.queue <- struct{}{}
return nil
}
func (p *ConnPool) remove(cn *Conn, reason error) {
p.storeLastErr(reason.Error())
for i, c := range p.conns {
if c == cn {
p.conns = append(p.conns[:i], p.conns[i+1:]...)
break
}
}
}
// Len returns total number of connections.
func (p *ConnPool) Len() int {
return p.conns.Len()
p.connsMu.Lock()
l := len(p.conns)
p.connsMu.Unlock()
return l
}
// FreeLen returns number of free connections.
func (p *ConnPool) FreeLen() int {
return p.freeConns.Len()
p.freeConnsMu.Lock()
l := len(p.freeConns)
p.freeConnsMu.Unlock()
return l
}
func (p *ConnPool) Stats() *PoolStats {
stats := p.stats
stats := PoolStats{}
stats.Requests = atomic.LoadUint32(&p.stats.Requests)
stats.Hits = atomic.LoadUint32(&p.stats.Hits)
stats.Waits = atomic.LoadUint32(&p.stats.Waits)
stats.Timeouts = atomic.LoadUint32(&p.stats.Timeouts)
stats.TotalConns = uint32(p.Len())
@ -269,16 +279,10 @@ func (p *ConnPool) Close() (retErr error) {
return ErrClosed
}
// Wait for app to free connections, but don't close them immediately.
for i := 0; i < p.Len()-p.FreeLen(); i++ {
if cn := p.wait(3 * time.Second); cn == nil {
break
}
}
p.connsMu.Lock()
// Close all connections.
cns := p.conns.Reset()
for _, cn := range cns {
for _, cn := range p.conns {
if cn == nil {
continue
}
@ -286,6 +290,12 @@ func (p *ConnPool) Close() (retErr error) {
retErr = err
}
}
p.conns = nil
p.connsMu.Unlock()
p.freeConnsMu.Lock()
p.freeConns = nil
p.freeConnsMu.Unlock()
return retErr
}
@ -298,16 +308,32 @@ func (p *ConnPool) closeConn(cn *Conn) error {
}
func (p *ConnPool) ReapStaleConns() (n int, err error) {
for {
cn := p.freeConns.ShiftStale(p.idleTimeout)
if cn == nil {
<-p.queue
p.freeConnsMu.Lock()
if len(p.freeConns) == 0 {
p.freeConnsMu.Unlock()
p.queue <- struct{}{}
return
}
var idx int
var cn *Conn
for idx, cn = range p.freeConns {
if !cn.IsStale(p.idleTimeout) {
break
}
if err = p.Remove(cn, errors.New("connection is stale")); err != nil {
return
}
p.connsMu.Lock()
p.remove(cn, errConnStale)
p.connsMu.Unlock()
n++
}
if idx > 0 {
p.freeConns = append(p.freeConns[:0], p.freeConns[idx:]...)
}
p.freeConnsMu.Unlock()
p.queue <- struct{}{}
return
}
@ -322,9 +348,13 @@ func (p *ConnPool) reaper(frequency time.Duration) {
n, err := p.ReapStaleConns()
if err != nil {
Logger.Printf("ReapStaleConns failed: %s", err)
} else if n > 0 {
Logger.Printf("removed %d stale connections", n)
continue
}
s := p.Stats()
Logger.Printf(
"reaper: removed %d stale conns (TotalConns=%d FreeConns=%d Requests=%d Hits=%d Timeouts=%d)",
n, s.TotalConns, s.FreeConns, s.Requests, s.Hits, s.Timeouts,
)
}
}

View File

@ -27,7 +27,7 @@ func (p *SingleConnPool) Put(cn *Conn) error {
return nil
}
func (p *SingleConnPool) Replace(cn *Conn, _ error) error {
func (p *SingleConnPool) Remove(cn *Conn, _ error) error {
if p.cn != cn {
panic("p.cn != cn")
}

View File

@ -67,13 +67,13 @@ func (p *StickyConnPool) Put(cn *Conn) error {
return nil
}
func (p *StickyConnPool) replace(reason error) error {
err := p.pool.Replace(p.cn, reason)
func (p *StickyConnPool) remove(reason error) error {
err := p.pool.Remove(p.cn, reason)
p.cn = nil
return err
}
func (p *StickyConnPool) Replace(cn *Conn, reason error) error {
func (p *StickyConnPool) Remove(cn *Conn, reason error) error {
defer p.mx.Unlock()
p.mx.Lock()
if p.closed {
@ -85,7 +85,7 @@ func (p *StickyConnPool) Replace(cn *Conn, reason error) error {
if cn != nil && p.cn != cn {
panic("p.cn != cn")
}
return p.replace(reason)
return p.remove(reason)
}
func (p *StickyConnPool) Len() int {
@ -121,7 +121,7 @@ func (p *StickyConnPool) Close() error {
err = p.put()
} else {
reason := errors.New("redis: sticky not reusable connection")
err = p.replace(reason)
err = p.remove(reason)
}
}
return err

View File

@ -16,8 +16,8 @@ var _ = Describe("ConnPool", func() {
var connPool *pool.ConnPool
BeforeEach(func() {
pool.SetIdleCheckFrequency(time.Second)
connPool = pool.NewConnPool(dummyDialer, 10, time.Hour, time.Second)
connPool = pool.NewConnPool(
dummyDialer, 10, time.Hour, time.Millisecond, time.Millisecond)
})
AfterEach(func() {
@ -33,7 +33,7 @@ var _ = Describe("ConnPool", func() {
break
}
_ = connPool.Replace(cn, errors.New("test"))
_ = connPool.Remove(cn, errors.New("test"))
}
Expect(rateErr).To(MatchError(`redis: you open connections too fast (last_error="test")`))
@ -75,7 +75,7 @@ var _ = Describe("ConnPool", func() {
// ok
}
err = connPool.Replace(cn, errors.New("test"))
err = connPool.Remove(cn, errors.New("test"))
Expect(err).NotTo(HaveOccurred())
// Check that Ping is unblocked.
@ -93,26 +93,33 @@ var _ = Describe("ConnPool", func() {
})
})
var _ = Describe("conns reapser", func() {
var _ = Describe("conns reaper", func() {
var connPool *pool.ConnPool
BeforeEach(func() {
pool.SetIdleCheckFrequency(time.Hour)
connPool = pool.NewConnPool(dummyDialer, 10, 0, time.Minute)
connPool = pool.NewConnPool(
dummyDialer, 10, time.Second, time.Millisecond, time.Hour)
var cns []*pool.Conn
// add stale connections
for i := 0; i < 3; i++ {
cn := pool.NewConn(&net.TCPConn{})
cn, err := connPool.Get()
Expect(err).NotTo(HaveOccurred())
cn.UsedAt = time.Now().Add(-2 * time.Minute)
Expect(connPool.Add(cn)).To(BeTrue())
Expect(cn.Index()).To(Equal(i))
cns = append(cns, cn)
}
// add fresh connections
for i := 0; i < 3; i++ {
cn := pool.NewConn(&net.TCPConn{})
Expect(connPool.Add(cn)).To(BeTrue())
Expect(cn.Index()).To(Equal(3 + i))
cn, err := connPool.Get()
Expect(err).NotTo(HaveOccurred())
cns = append(cns, cn)
}
for _, cn := range cns {
Expect(connPool.Put(cn)).NotTo(HaveOccurred())
}
Expect(connPool.Len()).To(Equal(6))
@ -136,7 +143,8 @@ var _ = Describe("conns reapser", func() {
for j := 0; j < 3; j++ {
var freeCns []*pool.Conn
for i := 0; i < 3; i++ {
cn := connPool.First()
cn, err := connPool.Get()
Expect(err).NotTo(HaveOccurred())
Expect(cn).NotTo(BeNil())
freeCns = append(freeCns, cn)
}
@ -144,9 +152,6 @@ var _ = Describe("conns reapser", func() {
Expect(connPool.Len()).To(Equal(3))
Expect(connPool.FreeLen()).To(Equal(0))
cn := connPool.First()
Expect(cn).To(BeNil())
cn, err := connPool.Get()
Expect(err).NotTo(HaveOccurred())
Expect(cn).NotTo(BeNil())
@ -173,42 +178,60 @@ var _ = Describe("conns reapser", func() {
var _ = Describe("race", func() {
var connPool *pool.ConnPool
var C, N = 10, 1000
if testing.Short() {
C = 4
N = 100
}
var C, N int
BeforeEach(func() {
pool.SetIdleCheckFrequency(time.Second)
connPool = pool.NewConnPool(dummyDialer, 10, time.Second, time.Second)
C, N = 10, 1000
if testing.Short() {
C = 4
N = 100
}
})
AfterEach(func() {
connPool.Close()
})
It("does not happend", func() {
It("does not happen on Get, Put, and Remove", func() {
connPool = pool.NewConnPool(
dummyDialer, 10, time.Minute, time.Millisecond, time.Millisecond)
connPool.DialLimiter = nil
perform(C, func(id int) {
for i := 0; i < N; i++ {
cn, err := connPool.Get()
Expect(err).NotTo(HaveOccurred())
if err == nil {
connPool.Put(cn)
Expect(connPool.Put(cn)).NotTo(HaveOccurred())
}
}
}, func(id int) {
for i := 0; i < N; i++ {
cn, err := connPool.Get()
Expect(err).NotTo(HaveOccurred())
if err == nil {
connPool.Replace(cn, errors.New("test"))
Expect(connPool.Remove(cn, errors.New("test"))).NotTo(HaveOccurred())
}
}
}, func(id int) {
})
})
It("does not happen on Get and PopFree", func() {
connPool = pool.NewConnPool(
dummyDialer, 10, time.Minute, time.Second, time.Millisecond)
connPool.DialLimiter = nil
perform(C, func(id int) {
for i := 0; i < N; i++ {
cn, err := connPool.Get()
Expect(err).NotTo(HaveOccurred())
if err == nil {
connPool.Remove(cn, errors.New("test"))
Expect(connPool.Put(cn)).NotTo(HaveOccurred())
}
cn = connPool.PopFree()
if cn != nil {
Expect(connPool.Put(cn)).NotTo(HaveOccurred())
}
}
})