mirror of
https://github.com/redis/go-redis.git
synced 2025-08-07 12:42:55 +03:00
Rework retrying
This commit is contained in:
252
cluster.go
252
cluster.go
@@ -13,10 +13,10 @@ import (
|
||||
"github.com/go-redis/redis/internal/hashtag"
|
||||
"github.com/go-redis/redis/internal/pool"
|
||||
"github.com/go-redis/redis/internal/proto"
|
||||
"github.com/go-redis/redis/internal/singleflight"
|
||||
)
|
||||
|
||||
var errClusterNoNodes = fmt.Errorf("redis: cluster has no nodes")
|
||||
var errNilClusterState = fmt.Errorf("redis: cannot load cluster slots")
|
||||
|
||||
// ClusterOptions are used to configure a cluster client and should be
|
||||
// passed to NewClusterClient.
|
||||
@@ -122,8 +122,8 @@ type clusterNode struct {
|
||||
Client *Client
|
||||
|
||||
latency uint32 // atomic
|
||||
generation uint32
|
||||
loading int64 // atomic
|
||||
generation uint32 // atomic
|
||||
loading int64 // atomic
|
||||
}
|
||||
|
||||
func newClusterNode(clOpt *ClusterOptions, addr string) *clusterNode {
|
||||
@@ -141,6 +141,14 @@ func newClusterNode(clOpt *ClusterOptions, addr string) *clusterNode {
|
||||
return &node
|
||||
}
|
||||
|
||||
func (n *clusterNode) Close() error {
|
||||
return n.Client.Close()
|
||||
}
|
||||
|
||||
func (n *clusterNode) Test() error {
|
||||
return n.Client.ClusterInfo().Err()
|
||||
}
|
||||
|
||||
func (n *clusterNode) updateLatency() {
|
||||
const probes = 10
|
||||
|
||||
@@ -154,14 +162,6 @@ func (n *clusterNode) updateLatency() {
|
||||
atomic.StoreUint32(&n.latency, latency)
|
||||
}
|
||||
|
||||
func (n *clusterNode) Close() error {
|
||||
return n.Client.Close()
|
||||
}
|
||||
|
||||
func (n *clusterNode) Test() error {
|
||||
return n.Client.ClusterInfo().Err()
|
||||
}
|
||||
|
||||
func (n *clusterNode) Latency() time.Duration {
|
||||
latency := atomic.LoadUint32(&n.latency)
|
||||
return time.Duration(latency) * time.Microsecond
|
||||
@@ -186,14 +186,16 @@ func (n *clusterNode) Loading() bool {
|
||||
}
|
||||
|
||||
func (n *clusterNode) Generation() uint32 {
|
||||
return n.generation
|
||||
return atomic.LoadUint32(&n.generation)
|
||||
}
|
||||
|
||||
func (n *clusterNode) SetGeneration(gen uint32) {
|
||||
if gen < n.generation {
|
||||
panic("gen < n.generation")
|
||||
for {
|
||||
v := atomic.LoadUint32(&n.generation)
|
||||
if gen < v || atomic.CompareAndSwapUint32(&n.generation, v, gen) {
|
||||
break
|
||||
}
|
||||
}
|
||||
n.generation = gen
|
||||
}
|
||||
|
||||
//------------------------------------------------------------------------------
|
||||
@@ -201,18 +203,23 @@ func (n *clusterNode) SetGeneration(gen uint32) {
|
||||
type clusterNodes struct {
|
||||
opt *ClusterOptions
|
||||
|
||||
mu sync.RWMutex
|
||||
addrs []string
|
||||
nodes map[string]*clusterNode
|
||||
closed bool
|
||||
mu sync.RWMutex
|
||||
allAddrs []string
|
||||
addrs []string
|
||||
nodes map[string]*clusterNode
|
||||
closed bool
|
||||
|
||||
nodeCreateGroup singleflight.Group
|
||||
|
||||
generation uint32
|
||||
}
|
||||
|
||||
func newClusterNodes(opt *ClusterOptions) *clusterNodes {
|
||||
return &clusterNodes{
|
||||
opt: opt,
|
||||
nodes: make(map[string]*clusterNode),
|
||||
opt: opt,
|
||||
|
||||
allAddrs: opt.Addrs,
|
||||
nodes: make(map[string]*clusterNode),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -231,6 +238,7 @@ func (c *clusterNodes) Close() error {
|
||||
firstErr = err
|
||||
}
|
||||
}
|
||||
|
||||
c.addrs = nil
|
||||
c.nodes = nil
|
||||
|
||||
@@ -238,9 +246,16 @@ func (c *clusterNodes) Close() error {
|
||||
}
|
||||
|
||||
func (c *clusterNodes) Addrs() ([]string, error) {
|
||||
var addrs []string
|
||||
c.mu.RLock()
|
||||
closed := c.closed
|
||||
addrs := c.addrs
|
||||
if !closed {
|
||||
if len(c.addrs) > 0 {
|
||||
addrs = c.addrs
|
||||
} else {
|
||||
addrs = c.allAddrs
|
||||
}
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
|
||||
if closed {
|
||||
@@ -310,6 +325,14 @@ func (c *clusterNodes) GetOrCreate(addr string) (*clusterNode, error) {
|
||||
return node, nil
|
||||
}
|
||||
|
||||
v, err := c.nodeCreateGroup.Do(addr, func() (interface{}, error) {
|
||||
node := newClusterNode(c.opt, addr)
|
||||
return node, node.Test()
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
@@ -319,12 +342,15 @@ func (c *clusterNodes) GetOrCreate(addr string) (*clusterNode, error) {
|
||||
|
||||
node, ok = c.nodes[addr]
|
||||
if ok {
|
||||
_ = v.(*clusterNode).Close()
|
||||
return node, nil
|
||||
}
|
||||
node = v.(*clusterNode)
|
||||
|
||||
c.allAddrs = appendIfNotExists(c.allAddrs, addr)
|
||||
c.addrs = append(c.addrs, addr)
|
||||
node = newClusterNode(c.opt, addr)
|
||||
c.nodes[addr] = node
|
||||
|
||||
return node, nil
|
||||
}
|
||||
|
||||
@@ -334,20 +360,8 @@ func (c *clusterNodes) Random() (*clusterNode, error) {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var nodeErr error
|
||||
for i := 0; i <= c.opt.MaxRedirects; i++ {
|
||||
n := rand.Intn(len(addrs))
|
||||
node, err := c.GetOrCreate(addrs[n])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
nodeErr = node.Test()
|
||||
if nodeErr == nil {
|
||||
return node, nil
|
||||
}
|
||||
}
|
||||
return nil, nodeErr
|
||||
n := rand.Intn(len(addrs))
|
||||
return c.GetOrCreate(addrs[n])
|
||||
}
|
||||
|
||||
//------------------------------------------------------------------------------
|
||||
@@ -470,9 +484,12 @@ func (c *clusterState) slotNodes(slot int) []*clusterNode {
|
||||
type ClusterClient struct {
|
||||
cmdable
|
||||
|
||||
opt *ClusterOptions
|
||||
nodes *clusterNodes
|
||||
_state atomic.Value
|
||||
opt *ClusterOptions
|
||||
nodes *clusterNodes
|
||||
|
||||
_state atomic.Value
|
||||
stateErrMu sync.RWMutex
|
||||
stateErr error
|
||||
|
||||
cmdsInfoOnce internal.Once
|
||||
cmdsInfo map[string]*CommandInfo
|
||||
@@ -501,20 +518,7 @@ func NewClusterClient(opt *ClusterOptions) *ClusterClient {
|
||||
|
||||
c.cmdable.setProcessor(c.Process)
|
||||
|
||||
// Add initial nodes.
|
||||
for _, addr := range opt.Addrs {
|
||||
_, _ = c.nodes.GetOrCreate(addr)
|
||||
}
|
||||
|
||||
// Preload cluster slots.
|
||||
for i := 0; i < 10; i++ {
|
||||
state, err := c.reloadState()
|
||||
if err == nil {
|
||||
c._state.Store(state)
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
c.reloadState()
|
||||
if opt.IdleCheckFrequency > 0 {
|
||||
go c.reaper(opt.IdleCheckFrequency)
|
||||
}
|
||||
@@ -531,21 +535,6 @@ func (c *ClusterClient) retryBackoff(attempt int) time.Duration {
|
||||
return internal.RetryBackoff(attempt, c.opt.MinRetryBackoff, c.opt.MaxRetryBackoff)
|
||||
}
|
||||
|
||||
func (c *ClusterClient) state() (*clusterState, error) {
|
||||
v := c._state.Load()
|
||||
if v != nil {
|
||||
return v.(*clusterState), nil
|
||||
}
|
||||
|
||||
_, err := c.nodes.Addrs()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
c.lazyReloadState()
|
||||
return nil, errNilClusterState
|
||||
}
|
||||
|
||||
func (c *ClusterClient) cmdInfo(name string) *CommandInfo {
|
||||
err := c.cmdsInfoOnce.Do(func() error {
|
||||
node, err := c.nodes.Random()
|
||||
@@ -584,7 +573,12 @@ func (c *ClusterClient) cmdSlot(cmd Cmder) int {
|
||||
return cmdSlot(cmd, cmdFirstKeyPos(cmd, cmdInfo))
|
||||
}
|
||||
|
||||
func (c *ClusterClient) cmdSlotAndNode(state *clusterState, cmd Cmder) (int, *clusterNode, error) {
|
||||
func (c *ClusterClient) cmdSlotAndNode(cmd Cmder) (int, *clusterNode, error) {
|
||||
state, err := c.state()
|
||||
if err != nil {
|
||||
return 0, nil, err
|
||||
}
|
||||
|
||||
cmdInfo := c.cmdInfo(cmd.Name())
|
||||
slot := cmdSlot(cmd, cmdFirstKeyPos(cmd, cmdInfo))
|
||||
|
||||
@@ -602,16 +596,24 @@ func (c *ClusterClient) cmdSlotAndNode(state *clusterState, cmd Cmder) (int, *cl
|
||||
return slot, node, err
|
||||
}
|
||||
|
||||
func (c *ClusterClient) slotMasterNode(slot int) (*clusterNode, error) {
|
||||
state, err := c.state()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
nodes := state.slotNodes(slot)
|
||||
if len(nodes) > 0 {
|
||||
return nodes[0], nil
|
||||
}
|
||||
return c.nodes.Random()
|
||||
}
|
||||
|
||||
func (c *ClusterClient) Watch(fn func(*Tx) error, keys ...string) error {
|
||||
if len(keys) == 0 {
|
||||
return fmt.Errorf("redis: keys don't hash to the same slot")
|
||||
}
|
||||
|
||||
state, err := c.state()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
slot := hashtag.Slot(keys[0])
|
||||
for _, key := range keys[1:] {
|
||||
if hashtag.Slot(key) != slot {
|
||||
@@ -619,7 +621,7 @@ func (c *ClusterClient) Watch(fn func(*Tx) error, keys ...string) error {
|
||||
}
|
||||
}
|
||||
|
||||
node, err := state.slotMasterNode(slot)
|
||||
node, err := c.slotMasterNode(slot)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -649,10 +651,11 @@ func (c *ClusterClient) Watch(fn func(*Tx) error, keys ...string) error {
|
||||
}
|
||||
|
||||
if err == pool.ErrClosed {
|
||||
node, err = state.slotMasterNode(slot)
|
||||
node, err = c.slotMasterNode(slot)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
return err
|
||||
@@ -683,13 +686,7 @@ func (c *ClusterClient) Process(cmd Cmder) error {
|
||||
}
|
||||
|
||||
func (c *ClusterClient) defaultProcess(cmd Cmder) error {
|
||||
state, err := c.state()
|
||||
if err != nil {
|
||||
cmd.setErr(err)
|
||||
return err
|
||||
}
|
||||
|
||||
_, node, err := c.cmdSlotAndNode(state, cmd)
|
||||
_, node, err := c.cmdSlotAndNode(cmd)
|
||||
if err != nil {
|
||||
cmd.setErr(err)
|
||||
return err
|
||||
@@ -747,11 +744,12 @@ func (c *ClusterClient) defaultProcess(cmd Cmder) error {
|
||||
}
|
||||
|
||||
if err == pool.ErrClosed {
|
||||
_, node, err = c.cmdSlotAndNode(state, cmd)
|
||||
_, node, err = c.cmdSlotAndNode(cmd)
|
||||
if err != nil {
|
||||
cmd.setErr(err)
|
||||
return err
|
||||
break
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
break
|
||||
@@ -903,31 +901,37 @@ func (c *ClusterClient) lazyReloadState() {
|
||||
if !atomic.CompareAndSwapUint32(&c.reloading, 0, 1) {
|
||||
return
|
||||
}
|
||||
|
||||
go func() {
|
||||
defer atomic.StoreUint32(&c.reloading, 0)
|
||||
|
||||
for {
|
||||
state, err := c.reloadState()
|
||||
if err == pool.ErrClosed {
|
||||
return
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
time.Sleep(time.Millisecond)
|
||||
continue
|
||||
}
|
||||
|
||||
c._state.Store(state)
|
||||
time.Sleep(5 * time.Second)
|
||||
c.nodes.GC(state.generation)
|
||||
break
|
||||
if c.reloadState() {
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
atomic.StoreUint32(&c.reloading, 0)
|
||||
}()
|
||||
}
|
||||
|
||||
// Not thread-safe.
|
||||
func (c *ClusterClient) reloadState() (*clusterState, error) {
|
||||
func (c *ClusterClient) reloadState() bool {
|
||||
for attempt := 0; attempt <= c.opt.MaxRedirects; attempt++ {
|
||||
if attempt > 0 {
|
||||
time.Sleep(c.retryBackoff(attempt))
|
||||
}
|
||||
|
||||
state, err := c.loadState()
|
||||
if err == nil {
|
||||
c._state.Store(state)
|
||||
c.nodes.GC(state.generation)
|
||||
return true
|
||||
}
|
||||
|
||||
c.setStateErr(err)
|
||||
switch err {
|
||||
case pool.ErrClosed, errClusterNoNodes:
|
||||
return false
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (c *ClusterClient) loadState() (*clusterState, error) {
|
||||
node, err := c.nodes.Random()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -941,6 +945,27 @@ func (c *ClusterClient) reloadState() (*clusterState, error) {
|
||||
return newClusterState(c.nodes, slots, node.Client.opt.Addr)
|
||||
}
|
||||
|
||||
func (c *ClusterClient) state() (*clusterState, error) {
|
||||
v := c._state.Load()
|
||||
if v != nil {
|
||||
return v.(*clusterState), nil
|
||||
}
|
||||
return nil, c.getStateErr()
|
||||
}
|
||||
|
||||
func (c *ClusterClient) setStateErr(err error) {
|
||||
c.stateErrMu.Lock()
|
||||
c.stateErr = err
|
||||
c.stateErrMu.Unlock()
|
||||
}
|
||||
|
||||
func (c *ClusterClient) getStateErr() error {
|
||||
c.stateErrMu.RLock()
|
||||
err := c.stateErr
|
||||
c.stateErrMu.RUnlock()
|
||||
return err
|
||||
}
|
||||
|
||||
// reaper closes idle connections to the cluster.
|
||||
func (c *ClusterClient) reaper(idleCheckFrequency time.Duration) {
|
||||
ticker := time.NewTicker(idleCheckFrequency)
|
||||
@@ -1055,15 +1080,17 @@ func (c *ClusterClient) remapCmds(cmds []Cmder, failedCmds map[*clusterNode][]Cm
|
||||
func (c *ClusterClient) pipelineProcessCmds(
|
||||
node *clusterNode, cn *pool.Conn, cmds []Cmder, failedCmds map[*clusterNode][]Cmder,
|
||||
) error {
|
||||
cn.SetWriteTimeout(c.opt.WriteTimeout)
|
||||
if err := writeCmd(cn, cmds...); err != nil {
|
||||
_ = cn.SetWriteTimeout(c.opt.WriteTimeout)
|
||||
|
||||
err := writeCmd(cn, cmds...)
|
||||
if err != nil {
|
||||
setCmdsErr(cmds, err)
|
||||
failedCmds[node] = cmds
|
||||
return err
|
||||
}
|
||||
|
||||
// Set read timeout for all commands.
|
||||
cn.SetReadTimeout(c.opt.ReadTimeout)
|
||||
_ = cn.SetReadTimeout(c.opt.ReadTimeout)
|
||||
|
||||
return c.pipelineReadCmds(cn, cmds, failedCmds)
|
||||
}
|
||||
@@ -1280,12 +1307,7 @@ func (c *ClusterClient) pubSub(channels []string) *PubSub {
|
||||
slot = -1
|
||||
}
|
||||
|
||||
state, err := c.state()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
masterNode, err := state.slotMasterNode(slot)
|
||||
masterNode, err := c.slotMasterNode(slot)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
Reference in New Issue
Block a user