5 Star 4 Fork 1

Gitee 极速下载/rueidis

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
文件
此仓库是为了提升国内下载速度的镜像仓库,每日同步一次。 原始仓库: https://github.com/rueian/rueidis
克隆/下载
cluster.go 34.96 KB
一键复制 编辑 原始数据 按行查看 历史
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431
package rueidis
import (
"context"
"errors"
"io"
"net"
"strconv"
"sync"
"sync/atomic"
"time"
"github.com/redis/rueidis/internal/cmds"
"github.com/redis/rueidis/internal/util"
)
// ErrNoSlot indicates that there is no redis node owns the key slot.
var ErrNoSlot = errors.New("the slot has no redis node")
var ErrReplicaOnlyConflict = errors.New("ReplicaOnly conflicts with SendToReplicas option")
var ErrInvalidShardsRefreshInterval = errors.New("ShardsRefreshInterval must be greater than or equal to 0")
var ErrReplicaOnlyConflictWithReplicaSelector = errors.New("ReplicaOnly conflicts with ReplicaSelector option")
var ErrSendToReplicasNotSet = errors.New("SendToReplicas must be set when ReplicaSelector is set")
type clusterClient struct {
pslots [16384]conn
rslots []conn
sc call
rOpt *ClientOption
conns map[string]connrole
connFn connFn
opt *ClientOption
retryHandler retryHandler
stopCh chan struct{}
cmd Builder
mu sync.RWMutex
stop uint32
retry bool
}
// NOTE: connrole and conn must be initialized at the same time
type connrole struct {
conn conn
hidden bool
//replica bool <- this field is removed because a server may have mixed roles at the same time in the future. https://github.com/valkey-io/valkey/issues/1372
}
var replicaOnlySelector = func(_ uint16, replicas []ReplicaInfo) int {
return util.FastRand(len(replicas))
}
func newClusterClient(opt *ClientOption, connFn connFn, retryer retryHandler) (*clusterClient, error) {
client := &clusterClient{
cmd: cmds.NewBuilder(cmds.InitSlot),
connFn: connFn,
opt: opt,
conns: make(map[string]connrole),
retry: !opt.DisableRetry,
retryHandler: retryer,
stopCh: make(chan struct{}),
}
if opt.ReplicaOnly && opt.SendToReplicas != nil {
return nil, ErrReplicaOnlyConflict
}
if opt.ReplicaOnly && opt.ReplicaSelector != nil {
return nil, ErrReplicaOnlyConflictWithReplicaSelector
}
if opt.ReplicaSelector != nil && opt.SendToReplicas == nil {
return nil, ErrSendToReplicasNotSet
}
if opt.SendToReplicas != nil && opt.ReplicaSelector == nil {
opt.ReplicaSelector = replicaOnlySelector
}
if opt.SendToReplicas != nil {
rOpt := *opt
rOpt.ReplicaOnly = true
client.rOpt = &rOpt
}
client.connFn = func(dst string, opt *ClientOption) conn {
cc := connFn(dst, opt)
cc.SetOnCloseHook(func(err error) {
client.lazyRefresh()
})
return cc
}
if err := client.init(); err != nil {
return nil, err
}
if err := client.refresh(context.Background()); err != nil {
return client, err
}
if opt.ClusterOption.ShardsRefreshInterval > 0 {
go client.runClusterTopologyRefreshment()
} else if opt.ClusterOption.ShardsRefreshInterval < 0 {
return nil, ErrInvalidShardsRefreshInterval
}
return client, nil
}
func (c *clusterClient) init() error {
if len(c.opt.InitAddress) == 0 {
return ErrNoAddr
}
results := make(chan error, len(c.opt.InitAddress))
for _, addr := range c.opt.InitAddress {
cc := c.connFn(addr, c.opt)
go func(addr string, cc conn) {
if err := cc.Dial(); err == nil {
c.mu.Lock()
if _, ok := c.conns[addr]; ok {
go cc.Close() // abort the new connection instead of closing the old one which may already been used
} else {
c.conns[addr] = connrole{
conn: cc,
}
}
c.mu.Unlock()
results <- nil
} else {
results <- err
}
}(addr, cc)
}
es := make([]error, cap(results))
for i := 0; i < cap(results); i++ {
if err := <-results; err == nil {
return nil
} else {
es[i] = err
}
}
return es[0]
}
func (c *clusterClient) refresh(ctx context.Context) (err error) {
return c.sc.Do(ctx, c._refresh)
}
func (c *clusterClient) lazyRefresh() {
c.sc.LazyDo(time.Second, c._refresh)
}
type clusterslots struct {
addr string
reply RedisResult
ver int
}
func (s clusterslots) parse(tls bool) map[string]group {
if s.ver < 8 {
return parseSlots(s.reply.val, s.addr)
}
return parseShards(s.reply.val, s.addr, tls)
}
func getClusterSlots(c conn, timeout time.Duration) clusterslots {
var ctx context.Context
var cancel context.CancelFunc
if timeout > 0 {
ctx, cancel = context.WithTimeout(context.Background(), timeout)
defer cancel()
} else {
ctx = context.Background()
}
v := c.Version()
if v < 8 {
return clusterslots{reply: c.Do(ctx, cmds.SlotCmd), addr: c.Addr(), ver: v}
}
return clusterslots{reply: c.Do(ctx, cmds.ShardsCmd), addr: c.Addr(), ver: v}
}
func (c *clusterClient) _refresh() (err error) {
c.mu.RLock()
results := make(chan clusterslots, len(c.conns))
pending := make([]conn, 0, len(c.conns))
for _, cc := range c.conns {
pending = append(pending, cc.conn)
}
c.mu.RUnlock()
var result clusterslots
for i := 0; i < cap(results); i++ {
if i&3 == 0 { // batch CLUSTER SLOTS/CLUSTER SHARDS for every 4 connections
for j := i; j < i+4 && j < len(pending); j++ {
go func(c conn, timeout time.Duration) {
results <- getClusterSlots(c, timeout)
}(pending[j], c.opt.ConnWriteTimeout)
}
}
result = <-results
err = result.reply.Error()
if len(result.reply.val.values) != 0 {
break
}
}
if err != nil {
return err
}
pending = nil
groups := result.parse(c.opt.TLSConfig != nil)
conns := make(map[string]connrole, len(groups))
for master, g := range groups {
conns[master] = connrole{conn: c.connFn(master, c.opt)}
if c.rOpt != nil {
for _, nodeInfo := range g.nodes[1:] {
conns[nodeInfo.Addr] = connrole{conn: c.connFn(nodeInfo.Addr, c.rOpt)}
}
} else {
for _, nodeInfo := range g.nodes[1:] {
conns[nodeInfo.Addr] = connrole{conn: c.connFn(nodeInfo.Addr, c.opt)}
}
}
}
// make sure InitAddress always be present
for _, addr := range c.opt.InitAddress {
if _, ok := conns[addr]; !ok {
conns[addr] = connrole{
conn: c.connFn(addr, c.opt),
hidden: true,
}
}
}
var removes []conn
c.mu.RLock()
for addr, cc := range c.conns {
if fresh, ok := conns[addr]; ok {
fresh.conn = cc.conn
conns[addr] = fresh
} else {
removes = append(removes, cc.conn)
}
}
c.mu.RUnlock()
pslots := [16384]conn{}
var rslots []conn
for master, g := range groups {
switch {
case c.opt.ReplicaOnly && len(g.nodes) > 1:
nodesCount := len(g.nodes)
for _, slot := range g.slots {
for i := slot[0]; i <= slot[1] && i >= 0 && i < 16384; i++ {
pslots[i] = conns[g.nodes[1+util.FastRand(nodesCount-1)].Addr].conn
}
}
case c.rOpt != nil:
if len(rslots) == 0 { // lazy init
rslots = make([]conn, 16384)
}
if len(g.nodes) > 1 {
n := len(g.nodes) - 1
for _, slot := range g.slots {
for i := slot[0]; i <= slot[1] && i >= 0 && i < 16384; i++ {
pslots[i] = conns[master].conn
rIndex := c.opt.ReplicaSelector(uint16(i), g.nodes[1:])
if rIndex >= 0 && rIndex < n {
rslots[i] = conns[g.nodes[1+rIndex].Addr].conn
} else {
rslots[i] = conns[master].conn
}
}
}
} else {
for _, slot := range g.slots {
for i := slot[0]; i <= slot[1] && i >= 0 && i < 16384; i++ {
pslots[i] = conns[master].conn
rslots[i] = conns[master].conn
}
}
}
default:
for _, slot := range g.slots {
for i := slot[0]; i <= slot[1] && i >= 0 && i < 16384; i++ {
pslots[i] = conns[master].conn
}
}
}
}
c.mu.Lock()
c.pslots = pslots
c.rslots = rslots
c.conns = conns
c.mu.Unlock()
if len(removes) > 0 {
go func(removes []conn) {
time.Sleep(time.Second * 5)
for _, cc := range removes {
cc.Close()
}
}(removes)
}
return nil
}
func (c *clusterClient) single() (conn conn) {
return c._pick(cmds.InitSlot, false)
}
func (c *clusterClient) nodes() []string {
c.mu.RLock()
nodes := make([]string, 0, len(c.conns))
for addr := range c.conns {
nodes = append(nodes, addr)
}
c.mu.RUnlock()
return nodes
}
type nodes []ReplicaInfo
type group struct {
nodes nodes
slots [][2]int64
}
func parseEndpoint(fallback, endpoint string, port int64) string {
switch endpoint {
case "":
endpoint, _, _ = net.SplitHostPort(fallback)
case "?":
return ""
}
return net.JoinHostPort(endpoint, strconv.FormatInt(port, 10))
}
// parseSlots - map redis slots for each redis nodes/addresses
// defaultAddr is needed in case the node does not know its own IP
func parseSlots(slots RedisMessage, defaultAddr string) map[string]group {
groups := make(map[string]group, len(slots.values))
for _, v := range slots.values {
master := parseEndpoint(defaultAddr, v.values[2].values[0].string, v.values[2].values[1].integer)
if master == "" {
continue
}
g, ok := groups[master]
if !ok {
g.slots = make([][2]int64, 0)
g.nodes = make(nodes, 0, len(v.values)-2)
for i := 2; i < len(v.values); i++ {
if dst := parseEndpoint(defaultAddr, v.values[i].values[0].string, v.values[i].values[1].integer); dst != "" {
g.nodes = append(g.nodes, ReplicaInfo{Addr: dst})
}
}
}
g.slots = append(g.slots, [2]int64{v.values[0].integer, v.values[1].integer})
groups[master] = g
}
return groups
}
// parseShards - map redis shards for each redis nodes/addresses
// defaultAddr is needed in case the node does not know its own IP
func parseShards(shards RedisMessage, defaultAddr string, tls bool) map[string]group {
groups := make(map[string]group, len(shards.values))
for _, v := range shards.values {
m := -1
shard, _ := v.AsMap()
slots := shard["slots"].values
_nodes := shard["nodes"].values
g := group{
nodes: make(nodes, 0, len(_nodes)),
slots: make([][2]int64, len(slots)/2),
}
for i := range g.slots {
g.slots[i][0], _ = slots[i*2].AsInt64()
g.slots[i][1], _ = slots[i*2+1].AsInt64()
}
for _, n := range _nodes {
dict, _ := n.AsMap()
if dict["health"].string != "online" {
continue
}
port := dict["port"].integer
if tls && dict["tls-port"].integer > 0 {
port = dict["tls-port"].integer
}
if dst := parseEndpoint(defaultAddr, dict["endpoint"].string, port); dst != "" {
if dict["role"].string == "master" {
m = len(g.nodes)
}
g.nodes = append(g.nodes, ReplicaInfo{Addr: dst})
}
}
if m >= 0 {
g.nodes[0], g.nodes[m] = g.nodes[m], g.nodes[0]
groups[g.nodes[0].Addr] = g
}
}
return groups
}
func (c *clusterClient) runClusterTopologyRefreshment() {
ticker := time.NewTicker(c.opt.ClusterOption.ShardsRefreshInterval)
defer ticker.Stop()
for {
select {
case <-c.stopCh:
return
case <-ticker.C:
c.lazyRefresh()
}
}
}
func (c *clusterClient) _pick(slot uint16, toReplica bool) (p conn) {
c.mu.RLock()
if slot == cmds.InitSlot {
for _, cc := range c.conns {
p = cc.conn
break
}
} else if toReplica && c.rslots != nil {
p = c.rslots[slot]
} else {
p = c.pslots[slot]
}
c.mu.RUnlock()
return p
}
func (c *clusterClient) pick(ctx context.Context, slot uint16, toReplica bool) (p conn, err error) {
if p = c._pick(slot, toReplica); p == nil {
if err := c.refresh(ctx); err != nil {
return nil, err
}
if p = c._pick(slot, toReplica); p == nil {
return nil, ErrNoSlot
}
}
return p, nil
}
func (c *clusterClient) redirectOrNew(addr string, prev conn, slot uint16, mode RedirectMode) conn {
c.mu.RLock()
cc := c.conns[addr]
c.mu.RUnlock()
if cc.conn != nil && prev != cc.conn {
return cc.conn
}
c.mu.Lock()
if cc = c.conns[addr]; cc.conn == nil {
p := c.connFn(addr, c.opt)
cc = connrole{conn: p}
c.conns[addr] = cc
if mode == RedirectMove {
c.pslots[slot] = p
}
} else if prev == cc.conn {
// try reconnection if the MOVED redirects to the same host,
// because the same hostname may actually be resolved into another destination
// depending on the fail-over implementation. ex: AWS MemoryDB's resize process.
go func(prev conn) {
time.Sleep(time.Second * 5)
prev.Close()
}(prev)
p := c.connFn(addr, c.opt)
cc = connrole{conn: p}
c.conns[addr] = cc
if mode == RedirectMove { // MOVED should always point to the primary.
c.pslots[slot] = p
}
}
c.mu.Unlock()
return cc.conn
}
func (c *clusterClient) B() Builder {
return c.cmd
}
func (c *clusterClient) Do(ctx context.Context, cmd Completed) (resp RedisResult) {
if resp = c.do(ctx, cmd); resp.NonRedisError() == nil { // not recycle cmds if error, since cmds may be used later in pipe. consider recycle them by pipe
cmds.PutCompleted(cmd)
}
return resp
}
func (c *clusterClient) do(ctx context.Context, cmd Completed) (resp RedisResult) {
attempts := 1
retry:
cc, err := c.pick(ctx, cmd.Slot(), c.toReplica(cmd))
if err != nil {
return newErrResult(err)
}
resp = cc.Do(ctx, cmd)
process:
switch addr, mode := c.shouldRefreshRetry(resp.Error(), ctx); mode {
case RedirectMove:
resp = c.redirectOrNew(addr, cc, cmd.Slot(), mode).Do(ctx, cmd)
goto process
case RedirectAsk:
results := c.redirectOrNew(addr, cc, cmd.Slot(), mode).DoMulti(ctx, cmds.AskingCmd, cmd)
resp = results.s[1]
resultsp.Put(results)
goto process
case RedirectRetry:
if c.retry && cmd.IsReadOnly() {
shouldRetry := c.retryHandler.WaitOrSkipRetry(ctx, attempts, cmd, resp.Error())
if shouldRetry {
attempts++
goto retry
}
}
}
return resp
}
func (c *clusterClient) toReplica(cmd Completed) bool {
if c.opt.SendToReplicas != nil {
return c.opt.SendToReplicas(cmd)
}
return false
}
func (c *clusterClient) _pickMulti(multi []Completed) (retries *connretry, init bool) {
last := cmds.InitSlot
for _, cmd := range multi {
if cmd.Slot() == cmds.InitSlot {
init = true
break
}
}
c.mu.RLock()
defer c.mu.RUnlock()
count := conncountp.Get(len(c.conns), len(c.conns))
if !init && c.rslots != nil && c.opt.SendToReplicas != nil {
for _, cmd := range multi {
var cc conn
if c.opt.SendToReplicas(cmd) {
cc = c.rslots[cmd.Slot()]
} else {
cc = c.pslots[cmd.Slot()]
}
if cc == nil {
return nil, false
}
count.m[cc]++
}
retries = connretryp.Get(len(count.m), len(count.m))
for cc, n := range count.m {
retries.m[cc] = retryp.Get(0, n)
}
conncountp.Put(count)
for i, cmd := range multi {
var cc conn
if c.opt.SendToReplicas(cmd) {
cc = c.rslots[cmd.Slot()]
} else {
cc = c.pslots[cmd.Slot()]
}
if cc == nil { // check cc == nil again in case of non-deterministic SendToReplicas.
return nil, false
}
re := retries.m[cc]
re.commands = append(re.commands, cmd)
re.cIndexes = append(re.cIndexes, i)
}
return retries, init
}
inits := 0
for _, cmd := range multi {
if cmd.Slot() == cmds.InitSlot {
inits++
continue
}
if last == cmds.InitSlot {
last = cmd.Slot()
} else if init && last != cmd.Slot() {
panic(panicMixCxSlot)
}
cc := c.pslots[cmd.Slot()]
if cc == nil {
return nil, false
}
count.m[cc]++
}
if last == cmds.InitSlot {
// if all commands have no slots, such as INFO, we pick a non-nil slot.
for i, cc := range c.pslots {
if cc != nil {
last = uint16(i)
count.m[cc] = inits
break
}
}
if last == cmds.InitSlot {
return nil, false
}
} else if init {
cc := c.pslots[last]
count.m[cc] += inits
}
retries = connretryp.Get(len(count.m), len(count.m))
for cc, n := range count.m {
retries.m[cc] = retryp.Get(0, n)
}
conncountp.Put(count)
for i, cmd := range multi {
var cc conn
if cmd.Slot() != cmds.InitSlot {
cc = c.pslots[cmd.Slot()]
} else {
cc = c.pslots[last]
}
re := retries.m[cc]
re.commands = append(re.commands, cmd)
re.cIndexes = append(re.cIndexes, i)
}
return retries, init
}
func (c *clusterClient) pickMulti(ctx context.Context, multi []Completed) (*connretry, bool, error) {
conns, hasInit := c._pickMulti(multi)
if conns == nil {
if err := c.refresh(ctx); err != nil {
return nil, false, err
}
if conns, hasInit = c._pickMulti(multi); conns == nil {
return nil, false, ErrNoSlot
}
}
return conns, hasInit, nil
}
func isMulti(cmd Completed) bool {
return len(cmd.Commands()) == 1 && cmd.Commands()[0] == "MULTI"
}
func isExec(cmd Completed) bool {
return len(cmd.Commands()) == 1 && cmd.Commands()[0] == "EXEC"
}
func (c *clusterClient) doresultfn(
ctx context.Context, results *redisresults, retries *connretry, mu *sync.Mutex, cc conn, cIndexes []int, commands []Completed, resps []RedisResult, attempts int, hasInit bool,
) (clean bool) {
mi := -1
ei := -1
clean = true
for i, resp := range resps {
clean = clean && resp.NonRedisError() == nil
ii := cIndexes[i]
cm := commands[i]
results.s[ii] = resp
addr, mode := c.shouldRefreshRetry(resp.Error(), ctx)
if mode != RedirectNone {
nc := cc
retryDelay := time.Duration(-1)
if mode == RedirectRetry {
if !c.retry || !cm.IsReadOnly() {
continue
}
retryDelay = c.retryHandler.RetryDelay(attempts, cm, resp.Error())
} else {
nc = c.redirectOrNew(addr, cc, cm.Slot(), mode)
}
if hasInit && ei < i { // find out if there is a transaction block or not.
for mi = i; mi >= 0 && !isMulti(commands[mi]) && !isExec(commands[mi]); mi-- {
}
for ei = i; ei < len(commands) && !isMulti(commands[ei]) && !isExec(commands[ei]); ei++ {
}
if mi >= 0 && ei < len(commands) && isMulti(commands[mi]) && isExec(commands[ei]) && resps[mi].val.string == ok { // a transaction is found.
mu.Lock()
retries.Redirects++
nr := retries.m[nc]
if nr == nil {
nr = retryp.Get(0, len(commands))
retries.m[nc] = nr
}
for i := mi; i <= ei; i++ {
ii := cIndexes[i]
cm := commands[i]
if mode == RedirectAsk {
nr.aIndexes = append(nr.aIndexes, ii)
nr.cAskings = append(nr.cAskings, cm)
} else {
nr.cIndexes = append(nr.cIndexes, ii)
nr.commands = append(nr.commands, cm)
}
}
mu.Unlock()
continue // the transaction has been added to the retries, go to the next cmd.
}
}
if hasInit && mi < i && i < ei && mi >= 0 && isMulti(commands[mi]) {
continue // the current cmd is in the processed transaction and has been added to the retries.
}
mu.Lock()
if mode != RedirectRetry {
retries.Redirects++
}
if mode == RedirectRetry && retryDelay >= 0 {
retries.RetryDelay = max(retries.RetryDelay, retryDelay)
}
nr := retries.m[nc]
if nr == nil {
nr = retryp.Get(0, len(commands))
retries.m[nc] = nr
}
if mode == RedirectAsk {
nr.aIndexes = append(nr.aIndexes, ii)
nr.cAskings = append(nr.cAskings, cm)
} else {
nr.cIndexes = append(nr.cIndexes, ii)
nr.commands = append(nr.commands, cm)
}
mu.Unlock()
}
}
return clean
}
func (c *clusterClient) doretry(
ctx context.Context, cc conn, results *redisresults, retries *connretry, re *retry, mu *sync.Mutex, wg *sync.WaitGroup, attempts int, hasInit bool,
) {
clean := true
if len(re.commands) != 0 {
resps := cc.DoMulti(ctx, re.commands...)
clean = c.doresultfn(ctx, results, retries, mu, cc, re.cIndexes, re.commands, resps.s, attempts, hasInit)
resultsp.Put(resps)
}
if len(re.cAskings) != 0 {
resps := askingMulti(cc, ctx, re.cAskings)
clean = c.doresultfn(ctx, results, retries, mu, cc, re.aIndexes, re.cAskings, resps.s, attempts, hasInit) && clean
resultsp.Put(resps)
}
if clean {
retryp.Put(re)
}
wg.Done()
}
func (c *clusterClient) DoMulti(ctx context.Context, multi ...Completed) []RedisResult {
if len(multi) == 0 {
return nil
}
retries, hasInit, err := c.pickMulti(ctx, multi)
if err != nil {
return fillErrs(len(multi), err)
}
defer connretryp.Put(retries)
var wg sync.WaitGroup
var mu sync.Mutex
results := resultsp.Get(len(multi), len(multi))
attempts := 1
retry:
retries.RetryDelay = -1 // Assume no retry. Because client retry flag can be set to false.
var cc1 conn
var re1 *retry
wg.Add(len(retries.m))
mu.Lock()
for cc, re := range retries.m {
delete(retries.m, cc)
cc1 = cc
re1 = re
break
}
for cc, re := range retries.m {
delete(retries.m, cc)
go c.doretry(ctx, cc, results, retries, re, &mu, &wg, attempts, hasInit)
}
mu.Unlock()
c.doretry(ctx, cc1, results, retries, re1, &mu, &wg, attempts, hasInit)
wg.Wait()
if len(retries.m) != 0 {
if retries.Redirects > 0 {
retries.Redirects = 0
goto retry
}
if retries.RetryDelay >= 0 {
c.retryHandler.WaitForRetry(ctx, retries.RetryDelay)
attempts++
goto retry
}
}
for i, cmd := range multi {
if results.s[i].NonRedisError() == nil {
cmds.PutCompleted(cmd)
}
}
return results.s
}
func fillErrs(n int, err error) (results []RedisResult) {
results = resultsp.Get(n, n).s
for i := range results {
results[i] = newErrResult(err)
}
return results
}
func (c *clusterClient) doCache(ctx context.Context, cmd Cacheable, ttl time.Duration) (resp RedisResult) {
attempts := 1
retry:
cc, err := c.pick(ctx, cmd.Slot(), c.toReplica(Completed(cmd)))
if err != nil {
return newErrResult(err)
}
resp = cc.DoCache(ctx, cmd, ttl)
process:
switch addr, mode := c.shouldRefreshRetry(resp.Error(), ctx); mode {
case RedirectMove:
resp = c.redirectOrNew(addr, cc, cmd.Slot(), mode).DoCache(ctx, cmd, ttl)
goto process
case RedirectAsk:
results := askingMultiCache(c.redirectOrNew(addr, cc, cmd.Slot(), mode), ctx, []CacheableTTL{CT(cmd, ttl)})
resp = results.s[0]
resultsp.Put(results)
goto process
case RedirectRetry:
if c.retry {
shouldRetry := c.retryHandler.WaitOrSkipRetry(ctx, attempts, Completed(cmd), resp.Error())
if shouldRetry {
attempts++
goto retry
}
}
}
return resp
}
func (c *clusterClient) DoCache(ctx context.Context, cmd Cacheable, ttl time.Duration) (resp RedisResult) {
resp = c.doCache(ctx, cmd, ttl)
if err := resp.NonRedisError(); err == nil || err == ErrDoCacheAborted {
cmds.PutCacheable(cmd)
}
return resp
}
func askingMulti(cc conn, ctx context.Context, multi []Completed) *redisresults {
var inTx bool
commands := make([]Completed, 0, len(multi)*2)
for _, cmd := range multi {
if inTx {
commands = append(commands, cmd)
inTx = !isExec(cmd)
} else {
commands = append(commands, cmds.AskingCmd, cmd)
inTx = isMulti(cmd)
}
}
results := resultsp.Get(0, len(multi))
resps := cc.DoMulti(ctx, commands...)
for i, resp := range resps.s {
if commands[i] != cmds.AskingCmd {
results.s = append(results.s, resp)
}
}
resultsp.Put(resps)
return results
}
func askingMultiCache(cc conn, ctx context.Context, multi []CacheableTTL) *redisresults {
commands := make([]Completed, 0, len(multi)*6)
for _, cmd := range multi {
ck, _ := cmds.CacheKey(cmd.Cmd)
commands = append(commands, cmds.OptInCmd, cmds.AskingCmd, cmds.MultiCmd, cmds.NewCompleted([]string{"PTTL", ck}), Completed(cmd.Cmd), cmds.ExecCmd)
}
results := resultsp.Get(0, len(multi))
resps := cc.DoMulti(ctx, commands...)
for i := 5; i < len(resps.s); i += 6 {
if arr, err := resps.s[i].ToArray(); err != nil {
if preErr := resps.s[i-1].Error(); preErr != nil { // if {Cmd} get a RedisError
err = preErr
}
results.s = append(results.s, newErrResult(err))
} else {
results.s = append(results.s, newResult(arr[len(arr)-1], nil))
}
}
resultsp.Put(resps)
return results
}
func (c *clusterClient) _pickMultiCache(multi []CacheableTTL) *connretrycache {
c.mu.RLock()
defer c.mu.RUnlock()
count := conncountp.Get(len(c.conns), len(c.conns))
if c.opt.SendToReplicas == nil || c.rslots == nil {
for _, cmd := range multi {
p := c.pslots[cmd.Cmd.Slot()]
if p == nil {
return nil
}
count.m[p]++
}
retries := connretrycachep.Get(len(count.m), len(count.m))
for cc, n := range count.m {
retries.m[cc] = retrycachep.Get(0, n)
}
conncountp.Put(count)
for i, cmd := range multi {
cc := c.pslots[cmd.Cmd.Slot()]
re := retries.m[cc]
re.commands = append(re.commands, cmd)
re.cIndexes = append(re.cIndexes, i)
}
return retries
} else {
for _, cmd := range multi {
var p conn
if c.opt.SendToReplicas(Completed(cmd.Cmd)) {
p = c.rslots[cmd.Cmd.Slot()]
} else {
p = c.pslots[cmd.Cmd.Slot()]
}
if p == nil {
return nil
}
count.m[p]++
}
retries := connretrycachep.Get(len(count.m), len(count.m))
for cc, n := range count.m {
retries.m[cc] = retrycachep.Get(0, n)
}
conncountp.Put(count)
for i, cmd := range multi {
var cc conn
if c.opt.SendToReplicas(Completed(cmd.Cmd)) {
cc = c.rslots[cmd.Cmd.Slot()]
} else {
cc = c.pslots[cmd.Cmd.Slot()]
}
re := retries.m[cc]
re.commands = append(re.commands, cmd)
re.cIndexes = append(re.cIndexes, i)
}
return retries
}
}
func (c *clusterClient) pickMultiCache(ctx context.Context, multi []CacheableTTL) (*connretrycache, error) {
conns := c._pickMultiCache(multi)
if conns == nil {
if err := c.refresh(ctx); err != nil {
return nil, err
}
if conns = c._pickMultiCache(multi); conns == nil {
return nil, ErrNoSlot
}
}
return conns, nil
}
func (c *clusterClient) resultcachefn(
ctx context.Context, results *redisresults, retries *connretrycache, mu *sync.Mutex, cc conn, cIndexes []int, commands []CacheableTTL, resps []RedisResult, attempts int,
) (clean bool) {
clean = true
for i, resp := range resps {
clean = clean && resp.NonRedisError() == nil
ii := cIndexes[i]
cm := commands[i]
results.s[ii] = resp
addr, mode := c.shouldRefreshRetry(resp.Error(), ctx)
if mode != RedirectNone {
nc := cc
retryDelay := time.Duration(-1)
if mode == RedirectRetry {
if !c.retry {
continue
}
retryDelay = c.retryHandler.RetryDelay(attempts, Completed(cm.Cmd), resp.Error())
} else {
nc = c.redirectOrNew(addr, cc, cm.Cmd.Slot(), mode)
}
mu.Lock()
if mode != RedirectRetry {
retries.Redirects++
}
if mode == RedirectRetry && retryDelay >= 0 {
retries.RetryDelay = max(retries.RetryDelay, retryDelay)
}
nr := retries.m[nc]
if nr == nil {
nr = retrycachep.Get(0, len(commands))
retries.m[nc] = nr
}
if mode == RedirectAsk {
nr.aIndexes = append(nr.aIndexes, ii)
nr.cAskings = append(nr.cAskings, cm)
} else {
nr.cIndexes = append(nr.cIndexes, ii)
nr.commands = append(nr.commands, cm)
}
mu.Unlock()
}
}
return clean
}
func (c *clusterClient) doretrycache(
ctx context.Context, cc conn, results *redisresults, retries *connretrycache, re *retrycache, mu *sync.Mutex, wg *sync.WaitGroup, attempts int,
) {
clean := true
if len(re.commands) != 0 {
resps := cc.DoMultiCache(ctx, re.commands...)
clean = c.resultcachefn(ctx, results, retries, mu, cc, re.cIndexes, re.commands, resps.s, attempts)
resultsp.Put(resps)
}
if len(re.cAskings) != 0 {
resps := askingMultiCache(cc, ctx, re.cAskings)
clean = c.resultcachefn(ctx, results, retries, mu, cc, re.aIndexes, re.cAskings, resps.s, attempts) && clean
resultsp.Put(resps)
}
if clean {
retrycachep.Put(re)
}
wg.Done()
}
func (c *clusterClient) DoMultiCache(ctx context.Context, multi ...CacheableTTL) []RedisResult {
if len(multi) == 0 {
return nil
}
retries, err := c.pickMultiCache(ctx, multi)
if err != nil {
return fillErrs(len(multi), err)
}
defer connretrycachep.Put(retries)
var wg sync.WaitGroup
var mu sync.Mutex
results := resultsp.Get(len(multi), len(multi))
attempts := 1
retry:
retries.RetryDelay = -1 // Assume no retry. Because client retry flag can be set to false.
var cc1 conn
var re1 *retrycache
wg.Add(len(retries.m))
mu.Lock()
for cc, re := range retries.m {
delete(retries.m, cc)
cc1 = cc
re1 = re
break
}
for cc, re := range retries.m {
delete(retries.m, cc)
go c.doretrycache(ctx, cc, results, retries, re, &mu, &wg, attempts)
}
mu.Unlock()
c.doretrycache(ctx, cc1, results, retries, re1, &mu, &wg, attempts)
wg.Wait()
if len(retries.m) != 0 {
if retries.Redirects > 0 {
retries.Redirects = 0
goto retry
}
if retries.RetryDelay >= 0 {
c.retryHandler.WaitForRetry(ctx, retries.RetryDelay)
attempts++
goto retry
}
}
for i, cmd := range multi {
if err := results.s[i].NonRedisError(); err == nil || err == ErrDoCacheAborted {
cmds.PutCacheable(cmd.Cmd)
}
}
return results.s
}
func (c *clusterClient) Receive(ctx context.Context, subscribe Completed, fn func(msg PubSubMessage)) (err error) {
attempts := 1
retry:
cc, err := c.pick(ctx, subscribe.Slot(), c.toReplica(subscribe))
if err != nil {
goto ret
}
err = cc.Receive(ctx, subscribe, fn)
if _, mode := c.shouldRefreshRetry(err, ctx); c.retry && mode != RedirectNone {
shouldRetry := c.retryHandler.WaitOrSkipRetry(ctx, attempts, subscribe, err)
if shouldRetry {
attempts++
goto retry
}
}
ret:
if err == nil {
cmds.PutCompleted(subscribe)
}
return err
}
func (c *clusterClient) DoStream(ctx context.Context, cmd Completed) RedisResultStream {
cc, err := c.pick(ctx, cmd.Slot(), c.toReplica(cmd))
if err != nil {
return RedisResultStream{e: err}
}
ret := cc.DoStream(ctx, cmd)
cmds.PutCompleted(cmd)
return ret
}
func (c *clusterClient) DoMultiStream(ctx context.Context, multi ...Completed) MultiRedisResultStream {
if len(multi) == 0 {
return RedisResultStream{e: io.EOF}
}
slot := multi[0].Slot()
repl := c.toReplica(multi[0])
for i := 1; i < len(multi); i++ {
if s := multi[i].Slot(); s != cmds.InitSlot {
if slot == cmds.InitSlot {
slot = s
} else if slot != s {
panic("DoMultiStream across multiple slots is not supported")
}
}
repl = repl && c.toReplica(multi[i])
}
cc, err := c.pick(ctx, slot, repl)
if err != nil {
return RedisResultStream{e: err}
}
ret := cc.DoMultiStream(ctx, multi...)
for _, cmd := range multi {
cmds.PutCompleted(cmd)
}
return ret
}
func (c *clusterClient) Dedicated(fn func(DedicatedClient) error) (err error) {
dcc := &dedicatedClusterClient{cmd: c.cmd, client: c, slot: cmds.NoSlot, retry: c.retry, retryHandler: c.retryHandler}
err = fn(dcc)
dcc.release()
return err
}
func (c *clusterClient) Dedicate() (DedicatedClient, func()) {
dcc := &dedicatedClusterClient{cmd: c.cmd, client: c, slot: cmds.NoSlot, retry: c.retry, retryHandler: c.retryHandler}
return dcc, dcc.release
}
func (c *clusterClient) Nodes() map[string]Client {
c.mu.RLock()
_nodes := make(map[string]Client, len(c.conns))
disableCache := c.opt != nil && c.opt.DisableCache
for addr, cc := range c.conns {
if !cc.hidden {
_nodes[addr] = newSingleClientWithConn(cc.conn, c.cmd, c.retry, disableCache, c.retryHandler)
}
}
c.mu.RUnlock()
return _nodes
}
func (c *clusterClient) Close() {
if atomic.CompareAndSwapUint32(&c.stop, 0, 1) {
close(c.stopCh)
}
c.mu.RLock()
for _, cc := range c.conns {
go cc.conn.Close()
}
c.mu.RUnlock()
}
func (c *clusterClient) shouldRefreshRetry(err error, ctx context.Context) (addr string, mode RedirectMode) {
if err != nil && err != Nil && err != ErrDoCacheAborted && atomic.LoadUint32(&c.stop) == 0 {
if err, ok := err.(*RedisError); ok {
if addr, ok = err.IsMoved(); ok {
mode = RedirectMove
} else if addr, ok = err.IsAsk(); ok {
mode = RedirectAsk
} else if err.IsClusterDown() || err.IsTryAgain() || err.IsLoading() {
mode = RedirectRetry
}
} else if ctx.Err() == nil {
mode = RedirectRetry
}
if mode != RedirectNone {
c.lazyRefresh()
}
}
return
}
type dedicatedClusterClient struct {
client *clusterClient
conn conn
wire wire
pshks *pshks
mu sync.Mutex
cmd Builder
retryHandler retryHandler
retry bool
slot uint16
mark bool
}
func (c *dedicatedClusterClient) acquire(ctx context.Context, slot uint16) (wire wire, err error) {
c.mu.Lock()
defer c.mu.Unlock()
if c.mark {
return nil, ErrDedicatedClientRecycled
}
if c.slot == cmds.NoSlot {
c.slot = slot
} else if c.slot != slot && slot != cmds.InitSlot {
panic(panicMsgCxSlot)
}
if c.wire != nil {
return c.wire, nil
}
if c.conn, err = c.client.pick(ctx, c.slot, false); err != nil {
if p := c.pshks; p != nil {
c.pshks = nil
p.close <- err
close(p.close)
}
return nil, err
}
c.wire = c.conn.Acquire()
if p := c.pshks; p != nil {
c.pshks = nil
ch := c.wire.SetPubSubHooks(p.hooks)
go func(ch <-chan error) {
for e := range ch {
p.close <- e
}
close(p.close)
}(ch)
}
return c.wire, nil
}
func (c *dedicatedClusterClient) release() {
c.mu.Lock()
if !c.mark {
if p := c.pshks; p != nil {
c.pshks = nil
close(p.close)
}
if c.wire != nil {
c.conn.Store(c.wire)
}
}
c.mark = true
c.mu.Unlock()
}
func (c *dedicatedClusterClient) B() Builder {
return c.cmd
}
func (c *dedicatedClusterClient) Do(ctx context.Context, cmd Completed) (resp RedisResult) {
attempts := 1
retry:
if w, err := c.acquire(ctx, cmd.Slot()); err != nil {
resp = newErrResult(err)
} else {
resp = w.Do(ctx, cmd)
switch _, mode := c.client.shouldRefreshRetry(resp.Error(), ctx); mode {
case RedirectRetry:
if c.retry && cmd.IsReadOnly() && w.Error() == nil {
shouldRetry := c.retryHandler.WaitOrSkipRetry(
ctx, attempts, cmd, resp.Error(),
)
if shouldRetry {
attempts++
goto retry
}
}
}
}
if resp.NonRedisError() == nil {
cmds.PutCompleted(cmd)
}
return resp
}
func (c *dedicatedClusterClient) DoMulti(ctx context.Context, multi ...Completed) (resp []RedisResult) {
if len(multi) == 0 {
return nil
}
slot := chooseSlot(multi)
if slot == cmds.NoSlot {
panic(panicMsgCxSlot)
}
retryable := c.retry
if retryable {
retryable = allReadOnly(multi)
}
attempts := 1
retry:
if w, err := c.acquire(ctx, slot); err == nil {
resp = w.DoMulti(ctx, multi...).s
for i, r := range resp {
_, mode := c.client.shouldRefreshRetry(r.Error(), ctx)
if mode == RedirectRetry && retryable && w.Error() == nil {
shouldRetry := c.retryHandler.WaitOrSkipRetry(
ctx, attempts, multi[i], r.Error(),
)
if shouldRetry {
attempts++
goto retry
}
}
if mode != RedirectNone {
break
}
}
} else {
resp = resultsp.Get(len(multi), len(multi)).s
for i := range resp {
resp[i] = newErrResult(err)
}
}
for i, cmd := range multi {
if resp[i].NonRedisError() == nil {
cmds.PutCompleted(cmd)
}
}
return resp
}
func (c *dedicatedClusterClient) Receive(ctx context.Context, subscribe Completed, fn func(msg PubSubMessage)) (err error) {
var (
w wire
attempts = 1
)
retry:
if w, err = c.acquire(ctx, subscribe.Slot()); err == nil {
err = w.Receive(ctx, subscribe, fn)
if _, mode := c.client.shouldRefreshRetry(err, ctx); c.retry && mode == RedirectRetry && w.Error() == nil {
shouldRetry := c.retryHandler.WaitOrSkipRetry(ctx, attempts, subscribe, err)
if shouldRetry {
attempts++
goto retry
}
}
}
if err == nil {
cmds.PutCompleted(subscribe)
}
return err
}
func (c *dedicatedClusterClient) SetPubSubHooks(hooks PubSubHooks) <-chan error {
c.mu.Lock()
defer c.mu.Unlock()
if c.mark {
ch := make(chan error, 1)
ch <- ErrDedicatedClientRecycled
return ch
}
if p := c.pshks; p != nil {
c.pshks = nil
close(p.close)
}
if c.wire != nil {
return c.wire.SetPubSubHooks(hooks)
}
if hooks.isZero() {
return nil
}
ch := make(chan error, 1)
c.pshks = &pshks{hooks: hooks, close: ch}
return ch
}
func (c *dedicatedClusterClient) Close() {
c.mu.Lock()
if p := c.pshks; p != nil {
c.pshks = nil
p.close <- ErrClosing
close(p.close)
}
if c.wire != nil {
c.wire.Close()
}
c.mu.Unlock()
c.release()
}
type RedirectMode int
const (
RedirectNone RedirectMode = iota
RedirectMove
RedirectAsk
RedirectRetry
panicMsgCxSlot = "cross slot command in Dedicated is prohibited"
panicMixCxSlot = "Mixing no-slot and cross slot commands in DoMulti is prohibited"
)
Loading...
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
Go
1
https://gitee.com/mirrors/rueidis.git
[email protected]:mirrors/rueidis.git
mirrors
rueidis
rueidis
main

搜索帮助