github.com/m3db/m3@v1.5.0/src/dbnode/client/connection_pool.go (about)

     1  // Copyright (c) 2016 Uber Technologies, Inc.
     2  //
     3  // Permission is hereby granted, free of charge, to any person obtaining a copy
     4  // of this software and associated documentation files (the "Software"), to deal
     5  // in the Software without restriction, including without limitation the rights
     6  // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
     7  // copies of the Software, and to permit persons to whom the Software is
     8  // furnished to do so, subject to the following conditions:
     9  //
    10  // The above copyright notice and this permission notice shall be included in
    11  // all copies or substantial portions of the Software.
    12  //
    13  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
    14  // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
    15  // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
    16  // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
    17  // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
    18  // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
    19  // THE SOFTWARE.
    20  
    21  package client
    22  
    23  import (
    24  	"errors"
    25  	"fmt"
    26  	"math"
    27  	"math/rand"
    28  	"sync"
    29  	"sync/atomic"
    30  	"time"
    31  
    32  	murmur3 "github.com/m3db/stackmurmur3/v2"
    33  	"github.com/uber-go/tally"
    34  	"github.com/uber/tchannel-go/thrift"
    35  	"go.uber.org/zap"
    36  
    37  	"github.com/m3db/m3/src/dbnode/generated/thrift/rpc"
    38  	"github.com/m3db/m3/src/dbnode/topology"
    39  )
    40  
    41  const (
    42  	channelName = "Client"
    43  )
    44  
    45  var (
    46  	errConnectionPoolClosed           = errors.New("connection pool closed")
    47  	errConnectionPoolHasNoConnections = newHostNotAvailableError(errors.New("connection pool has no connections"))
    48  	errNodeNotBootstrapped            = errors.New("node not bootstrapped")
    49  )
    50  
    51  type connPool struct {
    52  	sync.RWMutex
    53  
    54  	opts               Options
    55  	host               topology.Host
    56  	pool               []conn
    57  	poolLen            int64
    58  	used               int64
    59  	connectRand        rand.Source
    60  	healthCheckRand    rand.Source
    61  	healthCheckNewConn healthCheckFn
    62  	healthCheck        healthCheckFn
    63  	sleepConnect       sleepFn
    64  	sleepHealth        sleepFn
    65  	sleepHealthRetry   sleepFn
    66  	status             status
    67  	healthStatus       tally.Gauge
    68  }
    69  
    70  type conn struct {
    71  	channel Channel
    72  	client  rpc.TChanNode
    73  }
    74  
    75  // NewConnectionFn is a function that creates a connection.
    76  type NewConnectionFn func(
    77  	channelName string, addr string, opts Options,
    78  ) (Channel, rpc.TChanNode, error)
    79  
    80  type healthCheckFn func(client rpc.TChanNode, opts Options, checkBootstrapped bool) error
    81  
    82  type sleepFn func(t time.Duration)
    83  
    84  func newConnectionPool(host topology.Host, opts Options) connectionPool {
    85  	seed := int64(murmur3.StringSum32(host.Address()))
    86  
    87  	scope := opts.InstrumentOptions().
    88  		MetricsScope().
    89  		Tagged(map[string]string{
    90  			"hostID": host.ID(),
    91  		})
    92  
    93  	p := &connPool{
    94  		opts:               opts,
    95  		host:               host,
    96  		pool:               make([]conn, 0, opts.MaxConnectionCount()),
    97  		poolLen:            0,
    98  		connectRand:        rand.NewSource(seed),
    99  		healthCheckRand:    rand.NewSource(seed + 1),
   100  		healthCheckNewConn: healthCheck,
   101  		healthCheck:        healthCheck,
   102  		sleepConnect:       time.Sleep,
   103  		sleepHealth:        time.Sleep,
   104  		sleepHealthRetry:   time.Sleep,
   105  		healthStatus:       scope.Gauge("health-status"),
   106  	}
   107  
   108  	return p
   109  }
   110  
   111  func (p *connPool) Open() {
   112  	p.Lock()
   113  	defer p.Unlock()
   114  
   115  	if p.status != statusNotOpen {
   116  		return
   117  	}
   118  
   119  	p.status = statusOpen
   120  
   121  	connectEvery := p.opts.BackgroundConnectInterval()
   122  	connectStutter := p.opts.BackgroundConnectStutter()
   123  	go p.connectEvery(connectEvery, connectStutter)
   124  
   125  	healthCheckEvery := p.opts.BackgroundHealthCheckInterval()
   126  	healthCheckStutter := p.opts.BackgroundHealthCheckStutter()
   127  	go p.healthCheckEvery(healthCheckEvery, healthCheckStutter)
   128  }
   129  
   130  func (p *connPool) ConnectionCount() int {
   131  	p.RLock()
   132  	poolLen := p.poolLen
   133  	p.RUnlock()
   134  	return int(poolLen)
   135  }
   136  
   137  func (p *connPool) NextClient() (rpc.TChanNode, Channel, error) {
   138  	p.RLock()
   139  	if p.status != statusOpen {
   140  		p.RUnlock()
   141  		return nil, nil, errConnectionPoolClosed
   142  	}
   143  	if p.poolLen < 1 {
   144  		p.RUnlock()
   145  		return nil, nil, errConnectionPoolHasNoConnections
   146  	}
   147  	n := atomic.AddInt64(&p.used, 1)
   148  	conn := p.pool[n%p.poolLen]
   149  	p.RUnlock()
   150  	return conn.client, conn.channel, nil
   151  }
   152  
   153  func (p *connPool) Close() {
   154  	p.Lock()
   155  	if p.status != statusOpen {
   156  		p.Unlock()
   157  		return
   158  	}
   159  	p.status = statusClosed
   160  	p.Unlock()
   161  
   162  	for i := range p.pool {
   163  		p.pool[i].channel.Close()
   164  	}
   165  }
   166  
   167  func (p *connPool) connectEvery(interval time.Duration, stutter time.Duration) {
   168  	log := p.opts.InstrumentOptions().Logger()
   169  	target := p.opts.MaxConnectionCount()
   170  
   171  	for {
   172  		p.RLock()
   173  		state := p.status
   174  		poolLen := int(p.poolLen)
   175  		p.RUnlock()
   176  		if state != statusOpen {
   177  			return
   178  		}
   179  
   180  		address := p.host.Address()
   181  
   182  		var wg sync.WaitGroup
   183  		for i := 0; i < target-poolLen; i++ {
   184  			wg.Add(1)
   185  			newConnFn := p.opts.NewConnectionFn()
   186  			go func() {
   187  				defer wg.Done()
   188  
   189  				// Create connection
   190  				channel, client, err := newConnFn(channelName, address, p.opts)
   191  				if err != nil {
   192  					log.Debug("could not connect", zap.String("host", address), zap.Error(err))
   193  					return
   194  				}
   195  
   196  				// Health check the connection
   197  				if err := p.healthCheckNewConn(client, p.opts, false); err != nil {
   198  					p.maybeEmitHealthStatus(healthStatusCheckFailed)
   199  					log.Debug("could not connect, failed health check", zap.String("host", address), zap.Error(err))
   200  					channel.Close()
   201  					return
   202  				}
   203  
   204  				p.maybeEmitHealthStatus(healthStatusOK)
   205  				p.Lock()
   206  				if p.status == statusOpen {
   207  					p.pool = append(p.pool, conn{channel, client})
   208  					p.poolLen = int64(len(p.pool))
   209  				} else {
   210  					// NB(antanas): just being defensive.
   211  					// It's likely a corner case and happens only during server shutdown.
   212  					channel.Close()
   213  				}
   214  				p.Unlock()
   215  			}()
   216  		}
   217  
   218  		wg.Wait()
   219  
   220  		p.sleepConnect(interval + randStutter(p.connectRand, stutter))
   221  	}
   222  }
   223  
   224  func (p *connPool) maybeEmitHealthStatus(hs healthStatus) {
   225  	if p.opts.HostQueueEmitsHealthStatus() {
   226  		p.healthStatus.Update(float64(hs))
   227  	}
   228  }
   229  
   230  func (p *connPool) healthCheckEvery(interval time.Duration, stutter time.Duration) {
   231  	log := p.opts.InstrumentOptions().Logger()
   232  	nowFn := p.opts.ClockOptions().NowFn()
   233  
   234  	for {
   235  		p.RLock()
   236  		state := p.status
   237  		p.RUnlock()
   238  		if state != statusOpen {
   239  			return
   240  		}
   241  
   242  		var (
   243  			wg       sync.WaitGroup
   244  			start    = nowFn()
   245  			deadline = start.Add(interval + randStutter(p.healthCheckRand, stutter))
   246  		)
   247  
   248  		p.RLock()
   249  		for i := int64(0); i < p.poolLen; i++ {
   250  			wg.Add(1)
   251  			go func(client rpc.TChanNode) {
   252  				defer wg.Done()
   253  
   254  				var (
   255  					attempts = p.opts.BackgroundHealthCheckFailLimit()
   256  					failed   = 0
   257  					checkErr error
   258  				)
   259  				for j := 0; j < attempts; j++ {
   260  					if err := p.healthCheck(client, p.opts, false); err != nil {
   261  						checkErr = err
   262  						failed++
   263  						throttleDuration := time.Duration(math.Max(
   264  							float64(time.Second),
   265  							p.opts.BackgroundHealthCheckFailThrottleFactor()*
   266  								float64(p.opts.HostConnectTimeout())))
   267  						p.sleepHealthRetry(throttleDuration)
   268  						continue
   269  					}
   270  					// Healthy
   271  					break
   272  				}
   273  
   274  				healthy := failed < attempts
   275  				if !healthy {
   276  					// Log health check error
   277  					log.Debug("health check failed", zap.String("host", p.host.Address()), zap.Error(checkErr))
   278  
   279  					// Swap with tail and decrement pool size
   280  					p.Lock()
   281  					if p.status != statusOpen {
   282  						p.Unlock()
   283  						return
   284  					}
   285  					var c conn
   286  					for j := int64(0); j < p.poolLen; j++ {
   287  						if client == p.pool[j].client {
   288  							c = p.pool[j]
   289  							p.pool[j] = p.pool[p.poolLen-1]
   290  							p.pool = p.pool[:p.poolLen-1]
   291  							p.poolLen = int64(len(p.pool))
   292  							break
   293  						}
   294  					}
   295  					p.Unlock()
   296  
   297  					// Close the client's channel
   298  					c.channel.Close()
   299  				}
   300  			}(p.pool[i].client)
   301  		}
   302  		p.RUnlock()
   303  
   304  		wg.Wait()
   305  
   306  		now := nowFn()
   307  		if !now.Before(deadline) {
   308  			// Exceeded deadline, start next health check loop
   309  			p.sleepHealth(0) // Call sleep 0 for tests to intercept this loop continuation
   310  			continue
   311  		}
   312  
   313  		p.sleepHealth(deadline.Sub(now))
   314  	}
   315  }
   316  
   317  func healthCheck(client rpc.TChanNode, opts Options, checkBootstrapped bool) error {
   318  	tctx, _ := thrift.NewContext(opts.HostConnectTimeout())
   319  	result, err := client.Health(tctx)
   320  	if err != nil {
   321  		return err
   322  	}
   323  	if !result.Ok {
   324  		return fmt.Errorf("status not ok: %s", result.Status)
   325  	}
   326  	if checkBootstrapped && !result.Bootstrapped {
   327  		return errNodeNotBootstrapped
   328  	}
   329  	return nil
   330  }
   331  
   332  func randStutter(source rand.Source, t time.Duration) time.Duration {
   333  	amount := float64(source.Int63()) / float64(math.MaxInt64)
   334  	return time.Duration(float64(t) * amount)
   335  }