redis/ring.go

659 lines
14 KiB
Go
Raw Normal View History

2015-05-25 16:22:27 +03:00
package redis
import (
"context"
2015-05-25 16:22:27 +03:00
"errors"
"fmt"
"math/rand"
"strconv"
2015-05-25 16:22:27 +03:00
"sync"
"sync/atomic"
2015-05-25 16:22:27 +03:00
"time"
2017-02-18 17:42:34 +03:00
"github.com/go-redis/redis/internal"
"github.com/go-redis/redis/internal/consistenthash"
"github.com/go-redis/redis/internal/hashtag"
"github.com/go-redis/redis/internal/pool"
2015-05-25 16:22:27 +03:00
)
// Hash is type of hash function used in consistent hash.
type Hash consistenthash.Hash
2018-03-06 15:50:48 +03:00
2016-08-09 16:32:08 +03:00
var errRingShardsDown = errors.New("redis: all ring shards are down")
2015-05-25 16:22:27 +03:00
// RingOptions are used to configure a ring client and should be
// passed to NewRing.
type RingOptions struct {
2016-08-09 16:32:08 +03:00
// Map of name => host:port addresses of ring shards.
2015-05-25 16:22:27 +03:00
Addrs map[string]string
2016-08-09 16:32:08 +03:00
// Frequency of PING commands sent to check shards availability.
// Shard is considered down after 3 subsequent failed checks.
HeartbeatFrequency time.Duration
// Hash function used in consistent hash.
// Default is crc32.ChecksumIEEE.
Hash Hash
// Number of replicas in consistent hash.
// Default is 100 replicas.
//
// Higher number of replicas will provide less deviation, that is keys will be
// distributed to nodes more evenly.
//
// Following is deviation for common nreplicas:
// --------------------------------------------------------
// | nreplicas | standard error | 99% confidence interval |
// | 10 | 0.3152 | (0.37, 1.98) |
// | 100 | 0.0997 | (0.76, 1.28) |
// | 1000 | 0.0316 | (0.92, 1.09) |
// --------------------------------------------------------
//
// See https://arxiv.org/abs/1406.2294 for reference
HashReplicas int
2015-05-25 16:22:27 +03:00
// Following options are copied from Options struct.
2017-05-25 14:16:39 +03:00
OnConnect func(*Conn) error
DB int
2015-05-25 16:22:27 +03:00
Password string
2017-08-31 15:22:47 +03:00
MaxRetries int
MinRetryBackoff time.Duration
MaxRetryBackoff time.Duration
2015-06-04 11:50:24 +03:00
2015-05-25 16:22:27 +03:00
DialTimeout time.Duration
ReadTimeout time.Duration
WriteTimeout time.Duration
2016-03-17 19:00:47 +03:00
PoolSize int
2018-08-12 10:08:21 +03:00
MinIdleConns int
MaxConnAge time.Duration
2016-03-17 19:00:47 +03:00
PoolTimeout time.Duration
IdleTimeout time.Duration
IdleCheckFrequency time.Duration
2015-05-25 16:22:27 +03:00
}
2016-08-09 16:32:08 +03:00
func (opt *RingOptions) init() {
if opt.HeartbeatFrequency == 0 {
opt.HeartbeatFrequency = 500 * time.Millisecond
}
2017-08-31 15:22:47 +03:00
if opt.HashReplicas == 0 {
opt.HashReplicas = 100
}
2017-08-31 15:22:47 +03:00
switch opt.MinRetryBackoff {
case -1:
opt.MinRetryBackoff = 0
case 0:
opt.MinRetryBackoff = 8 * time.Millisecond
}
switch opt.MaxRetryBackoff {
case -1:
opt.MaxRetryBackoff = 0
case 0:
opt.MaxRetryBackoff = 512 * time.Millisecond
}
2016-08-09 16:32:08 +03:00
}
2016-06-05 14:10:30 +03:00
2015-05-25 16:22:27 +03:00
func (opt *RingOptions) clientOptions() *Options {
return &Options{
2017-05-25 14:16:39 +03:00
OnConnect: opt.OnConnect,
2015-05-25 16:22:27 +03:00
DB: opt.DB,
Password: opt.Password,
DialTimeout: opt.DialTimeout,
ReadTimeout: opt.ReadTimeout,
WriteTimeout: opt.WriteTimeout,
2016-03-17 19:00:47 +03:00
PoolSize: opt.PoolSize,
2018-08-12 10:08:21 +03:00
MinIdleConns: opt.MinIdleConns,
MaxConnAge: opt.MaxConnAge,
2016-03-17 19:00:47 +03:00
PoolTimeout: opt.PoolTimeout,
IdleTimeout: opt.IdleTimeout,
IdleCheckFrequency: opt.IdleCheckFrequency,
2015-05-25 16:22:27 +03:00
}
}
2018-03-07 15:08:40 +03:00
//------------------------------------------------------------------------------
2015-05-25 16:22:27 +03:00
type ringShard struct {
Client *Client
down int32
2015-05-25 16:22:27 +03:00
}
func (shard *ringShard) String() string {
var state string
if shard.IsUp() {
state = "up"
} else {
state = "down"
}
return fmt.Sprintf("%s is %s", shard.Client, state)
}
func (shard *ringShard) IsDown() bool {
2016-08-09 16:32:08 +03:00
const threshold = 3
return atomic.LoadInt32(&shard.down) >= threshold
2015-05-25 16:22:27 +03:00
}
func (shard *ringShard) IsUp() bool {
return !shard.IsDown()
}
// Vote votes to set shard state and returns true if state was changed.
func (shard *ringShard) Vote(up bool) bool {
if up {
changed := shard.IsDown()
atomic.StoreInt32(&shard.down, 0)
2015-05-25 16:22:27 +03:00
return changed
}
if shard.IsDown() {
return false
}
atomic.AddInt32(&shard.down, 1)
2015-05-25 16:22:27 +03:00
return shard.IsDown()
}
2018-03-07 15:08:40 +03:00
//------------------------------------------------------------------------------
type ringShards struct {
opt *RingOptions
2018-03-07 15:08:40 +03:00
mu sync.RWMutex
hash *consistenthash.Map
shards map[string]*ringShard // read only
list []*ringShard // read only
2018-07-23 15:55:13 +03:00
len int
2018-03-07 15:08:40 +03:00
closed bool
}
func newRingShards(opt *RingOptions) *ringShards {
2018-03-07 15:08:40 +03:00
return &ringShards{
2018-07-22 10:50:26 +03:00
opt: opt,
hash: newConsistentHash(opt),
2018-03-07 15:08:40 +03:00
shards: make(map[string]*ringShard),
}
}
func (c *ringShards) Add(name string, cl *Client) {
shard := &ringShard{Client: cl}
c.hash.Add(name)
c.shards[name] = shard
c.list = append(c.list, shard)
}
func (c *ringShards) List() []*ringShard {
c.mu.RLock()
list := c.list
c.mu.RUnlock()
return list
}
func (c *ringShards) Hash(key string) string {
c.mu.RLock()
hash := c.hash.Get(key)
c.mu.RUnlock()
return hash
}
func (c *ringShards) GetByKey(key string) (*ringShard, error) {
key = hashtag.Key(key)
c.mu.RLock()
if c.closed {
c.mu.RUnlock()
return nil, pool.ErrClosed
}
hash := c.hash.Get(key)
if hash == "" {
c.mu.RUnlock()
return nil, errRingShardsDown
}
shard := c.shards[hash]
c.mu.RUnlock()
return shard, nil
}
func (c *ringShards) GetByHash(name string) (*ringShard, error) {
if name == "" {
return c.Random()
}
c.mu.RLock()
shard := c.shards[name]
c.mu.RUnlock()
return shard, nil
}
func (c *ringShards) Random() (*ringShard, error) {
return c.GetByKey(strconv.Itoa(rand.Int()))
}
// heartbeat monitors state of each shard in the ring.
func (c *ringShards) Heartbeat(frequency time.Duration) {
ticker := time.NewTicker(frequency)
defer ticker.Stop()
for range ticker.C {
var rebalance bool
c.mu.RLock()
if c.closed {
c.mu.RUnlock()
break
}
shards := c.list
c.mu.RUnlock()
for _, shard := range shards {
err := shard.Client.Ping().Err()
if shard.Vote(err == nil || err == pool.ErrPoolTimeout) {
internal.Logf("ring shard state changed: %s", shard)
rebalance = true
}
}
if rebalance {
c.rebalance()
}
}
}
// rebalance removes dead shards from the Ring.
func (c *ringShards) rebalance() {
hash := newConsistentHash(c.opt)
2018-07-23 15:55:13 +03:00
var shardsNum int
2018-03-07 15:08:40 +03:00
for name, shard := range c.shards {
if shard.IsUp() {
hash.Add(name)
2018-07-23 15:55:13 +03:00
shardsNum++
2018-03-07 15:08:40 +03:00
}
}
c.mu.Lock()
c.hash = hash
2018-07-23 15:55:13 +03:00
c.len = shardsNum
2018-03-07 15:08:40 +03:00
c.mu.Unlock()
}
2018-07-23 15:55:13 +03:00
func (c *ringShards) Len() int {
c.mu.RLock()
l := c.len
c.mu.RUnlock()
return l
}
2018-03-07 15:08:40 +03:00
func (c *ringShards) Close() error {
c.mu.Lock()
defer c.mu.Unlock()
if c.closed {
return nil
}
c.closed = true
var firstErr error
for _, shard := range c.shards {
if err := shard.Client.Close(); err != nil && firstErr == nil {
firstErr = err
}
}
c.hash = nil
c.shards = nil
c.list = nil
return firstErr
}
//------------------------------------------------------------------------------
2015-05-25 16:22:27 +03:00
// Ring is a Redis client that uses constistent hashing to distribute
2015-09-12 09:36:03 +03:00
// keys across multiple Redis servers (shards). It's safe for
// concurrent use by multiple goroutines.
2015-05-25 16:22:27 +03:00
//
2015-11-21 11:20:01 +03:00
// Ring monitors the state of each shard and removes dead shards from
2016-07-02 11:07:27 +03:00
// the ring. When shard comes online it is added back to the ring. This
2015-05-25 16:22:27 +03:00
// gives you maximum availability and partition tolerance, but no
// consistency between different shards or even clients. Each client
// uses shards that are available to the client and does not do any
// coordination when shard state is changed.
//
2016-08-09 16:32:08 +03:00
// Ring should be used when you need multiple Redis servers for caching
2015-05-25 16:22:27 +03:00
// and can tolerate losing data when one of the servers dies.
// Otherwise you should use Redis Cluster.
type Ring struct {
cmdable
ctx context.Context
2018-03-06 15:50:48 +03:00
opt *RingOptions
2018-03-07 15:08:40 +03:00
shards *ringShards
2018-03-06 15:50:48 +03:00
cmdsInfoCache *cmdsInfoCache
2015-05-25 16:22:27 +03:00
2018-09-07 11:45:56 +03:00
process func(Cmder) error
2018-01-20 13:26:33 +03:00
processPipeline func([]Cmder) error
2015-05-25 16:22:27 +03:00
}
func NewRing(opt *RingOptions) *Ring {
2016-06-05 14:10:30 +03:00
opt.init()
2018-01-20 13:26:33 +03:00
2015-05-25 16:22:27 +03:00
ring := &Ring{
2018-05-17 15:21:51 +03:00
opt: opt,
shards: newRingShards(opt),
2015-05-25 16:22:27 +03:00
}
2018-05-17 15:21:51 +03:00
ring.cmdsInfoCache = newCmdsInfoCache(ring.cmdsInfo)
2018-09-07 11:45:56 +03:00
ring.process = ring.defaultProcess
2018-01-20 13:26:33 +03:00
ring.processPipeline = ring.defaultProcessPipeline
ring.cmdable.setProcessor(ring.Process)
2015-05-25 16:22:27 +03:00
for name, addr := range opt.Addrs {
clopt := opt.clientOptions()
clopt.Addr = addr
2018-03-07 15:08:40 +03:00
ring.shards.Add(name, NewClient(clopt))
2015-05-25 16:22:27 +03:00
}
2018-01-20 13:26:33 +03:00
2018-03-07 15:08:40 +03:00
go ring.shards.Heartbeat(opt.HeartbeatFrequency)
2018-01-20 13:26:33 +03:00
2015-05-25 16:22:27 +03:00
return ring
}
func (c *Ring) Context() context.Context {
if c.ctx != nil {
return c.ctx
}
return context.Background()
}
func (c *Ring) WithContext(ctx context.Context) *Ring {
if ctx == nil {
panic("nil context")
}
c2 := c.copy()
c2.ctx = ctx
return c2
}
2018-03-07 15:08:40 +03:00
func (c *Ring) copy() *Ring {
cp := *c
return &cp
}
2017-03-20 13:15:21 +03:00
// Options returns read-only Options that were used to create the client.
func (c *Ring) Options() *RingOptions {
return c.opt
}
2017-08-31 15:22:47 +03:00
func (c *Ring) retryBackoff(attempt int) time.Duration {
return internal.RetryBackoff(attempt, c.opt.MinRetryBackoff, c.opt.MaxRetryBackoff)
}
// PoolStats returns accumulated connection pool stats.
func (c *Ring) PoolStats() *PoolStats {
2018-03-07 15:08:40 +03:00
shards := c.shards.List()
var acc PoolStats
for _, shard := range shards {
s := shard.Client.connPool.Stats()
acc.Hits += s.Hits
acc.Misses += s.Misses
acc.Timeouts += s.Timeouts
acc.TotalConns += s.TotalConns
2018-08-12 10:08:21 +03:00
acc.IdleConns += s.IdleConns
}
return &acc
}
2018-07-23 15:55:13 +03:00
// Len returns the current number of shards in the ring.
func (c *Ring) Len() int {
return c.shards.Len()
}
2017-04-11 17:29:31 +03:00
// Subscribe subscribes the client to the specified channels.
func (c *Ring) Subscribe(channels ...string) *PubSub {
if len(channels) == 0 {
panic("at least one channel is required")
}
2018-03-07 15:08:40 +03:00
shard, err := c.shards.GetByKey(channels[0])
2017-04-11 17:29:31 +03:00
if err != nil {
// TODO: return PubSub with sticky error
panic(err)
}
return shard.Client.Subscribe(channels...)
}
// PSubscribe subscribes the client to the given patterns.
func (c *Ring) PSubscribe(channels ...string) *PubSub {
if len(channels) == 0 {
panic("at least one channel is required")
}
2018-03-07 15:08:40 +03:00
shard, err := c.shards.GetByKey(channels[0])
2017-04-11 17:29:31 +03:00
if err != nil {
// TODO: return PubSub with sticky error
panic(err)
}
return shard.Client.PSubscribe(channels...)
}
// ForEachShard concurrently calls the fn on each live shard in the ring.
// It returns the first error if any.
func (c *Ring) ForEachShard(fn func(client *Client) error) error {
2018-03-07 15:08:40 +03:00
shards := c.shards.List()
var wg sync.WaitGroup
errCh := make(chan error, 1)
for _, shard := range shards {
if shard.IsDown() {
continue
}
wg.Add(1)
go func(shard *ringShard) {
defer wg.Done()
err := fn(shard.Client)
if err != nil {
select {
case errCh <- err:
default:
}
}
}(shard)
}
wg.Wait()
select {
case err := <-errCh:
return err
default:
return nil
}
}
2018-05-17 15:21:51 +03:00
func (c *Ring) cmdsInfo() (map[string]*CommandInfo, error) {
shards := c.shards.List()
firstErr := errRingShardsDown
for _, shard := range shards {
cmdsInfo, err := shard.Client.Command().Result()
if err == nil {
return cmdsInfo, nil
}
2018-05-17 15:21:51 +03:00
if firstErr == nil {
firstErr = err
}
}
return nil, firstErr
}
func (c *Ring) cmdInfo(name string) *CommandInfo {
cmdsInfo, err := c.cmdsInfoCache.Get()
2017-06-17 12:34:39 +03:00
if err != nil {
return nil
}
2018-03-06 15:50:48 +03:00
info := cmdsInfo[name]
2017-08-31 15:22:47 +03:00
if info == nil {
internal.Logf("info for cmd=%s not found", name)
}
return info
}
func (c *Ring) cmdShard(cmd Cmder) (*ringShard, error) {
2017-05-09 12:44:36 +03:00
cmdInfo := c.cmdInfo(cmd.Name())
pos := cmdFirstKeyPos(cmd, cmdInfo)
if pos == 0 {
2018-03-07 15:08:40 +03:00
return c.shards.Random()
}
firstKey := cmd.stringArg(pos)
2018-03-07 15:08:40 +03:00
return c.shards.GetByKey(firstKey)
}
2018-08-12 11:11:01 +03:00
// Do creates a Cmd from the args and processes the cmd.
func (c *Ring) Do(args ...interface{}) *Cmd {
cmd := NewCmd(args...)
c.Process(cmd)
return cmd
}
func (c *Ring) WrapProcess(
fn func(oldProcess func(cmd Cmder) error) func(cmd Cmder) error,
) {
2018-09-07 11:45:56 +03:00
c.process = fn(c.process)
2018-01-20 13:26:33 +03:00
}
func (c *Ring) Process(cmd Cmder) error {
2018-09-07 11:45:56 +03:00
return c.process(cmd)
}
func (c *Ring) defaultProcess(cmd Cmder) error {
for attempt := 0; attempt <= c.opt.MaxRetries; attempt++ {
if attempt > 0 {
time.Sleep(c.retryBackoff(attempt))
}
shard, err := c.cmdShard(cmd)
if err != nil {
cmd.setErr(err)
return err
}
err = shard.Client.Process(cmd)
if err == nil {
return nil
}
if !internal.IsRetryableError(err, cmd.readTimeout() == nil) {
return err
}
2015-05-25 16:22:27 +03:00
}
2018-09-07 11:45:56 +03:00
return cmd.Err()
2015-05-25 16:22:27 +03:00
}
2017-05-02 18:00:53 +03:00
func (c *Ring) Pipeline() Pipeliner {
pipe := Pipeline{
2018-01-20 13:26:33 +03:00
exec: c.processPipeline,
2015-06-04 11:50:24 +03:00
}
2018-01-20 13:26:33 +03:00
pipe.cmdable.setProcessor(pipe.Process)
return &pipe
2015-06-04 11:50:24 +03:00
}
2017-05-02 18:00:53 +03:00
func (c *Ring) Pipelined(fn func(Pipeliner) error) ([]Cmder, error) {
2017-09-25 11:48:44 +03:00
return c.Pipeline().Pipelined(fn)
2015-06-04 11:50:24 +03:00
}
2018-01-20 13:26:33 +03:00
func (c *Ring) WrapProcessPipeline(
fn func(oldProcess func([]Cmder) error) func([]Cmder) error,
) {
c.processPipeline = fn(c.processPipeline)
}
func (c *Ring) defaultProcessPipeline(cmds []Cmder) error {
2015-06-04 11:50:24 +03:00
cmdsMap := make(map[string][]Cmder)
for _, cmd := range cmds {
2017-05-09 12:44:36 +03:00
cmdInfo := c.cmdInfo(cmd.Name())
2018-03-07 15:08:40 +03:00
hash := cmd.stringArg(cmdFirstKeyPos(cmd, cmdInfo))
if hash != "" {
hash = c.shards.Hash(hashtag.Key(hash))
}
2018-03-07 15:08:40 +03:00
cmdsMap[hash] = append(cmdsMap[hash], cmd)
2015-06-04 11:50:24 +03:00
}
2017-08-31 15:22:47 +03:00
for attempt := 0; attempt <= c.opt.MaxRetries; attempt++ {
if attempt > 0 {
time.Sleep(c.retryBackoff(attempt))
}
2018-09-11 13:09:16 +03:00
var mu sync.Mutex
var failedCmdsMap map[string][]Cmder
2018-09-11 13:09:16 +03:00
var wg sync.WaitGroup
2015-06-04 11:50:24 +03:00
2018-03-07 15:08:40 +03:00
for hash, cmds := range cmdsMap {
2018-09-11 13:09:16 +03:00
wg.Add(1)
go func(hash string, cmds []Cmder) {
defer wg.Done()
shard, err := c.shards.GetByHash(hash)
if err != nil {
setCmdsErr(cmds, err)
return
}
2015-06-04 11:50:24 +03:00
2018-09-11 13:09:16 +03:00
cn, err := shard.Client.getConn()
if err != nil {
setCmdsErr(cmds, err)
return
}
2017-08-31 15:22:47 +03:00
2018-09-11 13:09:16 +03:00
canRetry, err := shard.Client.pipelineProcessCmds(cn, cmds)
2018-10-14 11:27:34 +03:00
shard.Client.releaseConnStrict(cn, err)
2018-09-11 13:09:16 +03:00
if canRetry && internal.IsRetryableError(err, true) {
mu.Lock()
if failedCmdsMap == nil {
failedCmdsMap = make(map[string][]Cmder)
}
failedCmdsMap[hash] = cmds
mu.Unlock()
}
}(hash, cmds)
2015-06-04 11:50:24 +03:00
}
2018-09-11 13:09:16 +03:00
wg.Wait()
2015-06-04 11:50:24 +03:00
if len(failedCmdsMap) == 0 {
break
}
cmdsMap = failedCmdsMap
}
2018-08-12 11:11:01 +03:00
return cmdsFirstErr(cmds)
2015-06-04 11:50:24 +03:00
}
2017-09-25 11:48:44 +03:00
func (c *Ring) TxPipeline() Pipeliner {
panic("not implemented")
}
func (c *Ring) TxPipelined(fn func(Pipeliner) error) ([]Cmder, error) {
panic("not implemented")
}
2018-03-07 15:08:40 +03:00
// Close closes the ring client, releasing any open resources.
//
// It is rare to Close a Ring, as the Ring is meant to be long-lived
// and shared between many goroutines.
func (c *Ring) Close() error {
return c.shards.Close()
}
func newConsistentHash(opt *RingOptions) *consistenthash.Map {
return consistenthash.New(opt.HashReplicas, consistenthash.Hash(opt.Hash))
}