Add latency based routing to Redis Cluster client.

This commit is contained in:
Joris Minjat 2016-05-06 11:12:31 -07:00 committed by Vladimir Mihailenco
parent 3972f28066
commit 487feebef1
11 changed files with 760 additions and 509 deletions

View File

@ -11,6 +11,12 @@ import (
"gopkg.in/redis.v4/internal/pool" "gopkg.in/redis.v4/internal/pool"
) )
type clusterNode struct {
Addr string
Latency int
Client *Client
}
// ClusterClient is a Redis Cluster client representing a pool of zero // ClusterClient is a Redis Cluster client representing a pool of zero
// or more underlying connections. It's safe for concurrent use by // or more underlying connections. It's safe for concurrent use by
// multiple goroutines. // multiple goroutines.
@ -19,14 +25,14 @@ type ClusterClient struct {
opt *ClusterOptions opt *ClusterOptions
slotsMx sync.RWMutex // protects slots and addrs mu sync.RWMutex
addrs []string addrs []string
slots [][]string nodes map[string]*clusterNode
slots [][]*clusterNode
closed bool
clientsMx sync.RWMutex // protects clients and closed cmdsInfo map[string]*CommandInfo
clients map[string]*Client cmdsInfoOnce *sync.Once
_closed int32 // atomic
// Reports where slots reloading is in progress. // Reports where slots reloading is in progress.
reloading uint32 reloading uint32
@ -35,44 +41,63 @@ type ClusterClient struct {
// NewClusterClient returns a Redis Cluster client as described in // NewClusterClient returns a Redis Cluster client as described in
// http://redis.io/topics/cluster-spec. // http://redis.io/topics/cluster-spec.
func NewClusterClient(opt *ClusterOptions) *ClusterClient { func NewClusterClient(opt *ClusterOptions) *ClusterClient {
if opt.RouteByLatency {
opt.ReadOnly = true
}
client := &ClusterClient{ client := &ClusterClient{
opt: opt, opt: opt,
addrs: opt.Addrs, nodes: make(map[string]*clusterNode),
slots: make([][]string, hashtag.SlotNumber),
clients: make(map[string]*Client), cmdsInfoOnce: new(sync.Once),
} }
client.commandable.process = client.process client.commandable.process = client.process
for _, addr := range opt.Addrs {
_ = client.nodeByAddr(addr)
}
client.reloadSlots() client.reloadSlots()
return client return client
} }
// getClients returns a snapshot of clients for cluster nodes func (c *ClusterClient) cmdInfo(name string) *CommandInfo {
// this ClusterClient has been working with recently. c.cmdsInfoOnce.Do(func() {
// Note that snapshot can contain closed clients. for _, node := range c.nodes {
func (c *ClusterClient) getClients() map[string]*Client { cmdsInfo, err := node.Client.Command().Result()
c.clientsMx.RLock() if err == nil {
clients := make(map[string]*Client, len(c.clients)) c.cmdsInfo = cmdsInfo
for addr, client := range c.clients { return
clients[addr] = client }
}
c.cmdsInfoOnce = &sync.Once{}
})
return c.cmdsInfo[name]
}
func (c *ClusterClient) getNodes() map[string]*clusterNode {
c.mu.RLock()
var nodes map[string]*clusterNode
if !c.closed {
nodes = make(map[string]*clusterNode, len(c.nodes))
for addr, node := range c.nodes {
nodes[addr] = node
}
} }
c.clientsMx.RUnlock() c.mu.RUnlock()
return clients return nodes
} }
func (c *ClusterClient) Watch(fn func(*Tx) error, keys ...string) error { func (c *ClusterClient) Watch(fn func(*Tx) error, keys ...string) error {
addr := c.slotMasterAddr(hashtag.Slot(keys[0])) node := c.slotMasterNode(hashtag.Slot(keys[0]))
client, err := c.getClient(addr) return node.Client.Watch(fn, keys...)
if err != nil {
return err
}
return client.Watch(fn, keys...)
} }
// PoolStats returns accumulated connection pool stats. // PoolStats returns accumulated connection pool stats.
func (c *ClusterClient) PoolStats() *PoolStats { func (c *ClusterClient) PoolStats() *PoolStats {
acc := PoolStats{} acc := PoolStats{}
for _, client := range c.getClients() { for _, node := range c.getNodes() {
s := client.connPool.Stats() s := node.Client.connPool.Stats()
acc.Requests += s.Requests acc.Requests += s.Requests
acc.Hits += s.Hits acc.Hits += s.Hits
acc.Timeouts += s.Timeouts acc.Timeouts += s.Timeouts
@ -82,113 +107,163 @@ func (c *ClusterClient) PoolStats() *PoolStats {
return &acc return &acc
} }
func (c *ClusterClient) closed() bool {
return atomic.LoadInt32(&c._closed) == 1
}
// Close closes the cluster client, releasing any open resources. // Close closes the cluster client, releasing any open resources.
// //
// It is rare to Close a ClusterClient, as the ClusterClient is meant // It is rare to Close a ClusterClient, as the ClusterClient is meant
// to be long-lived and shared between many goroutines. // to be long-lived and shared between many goroutines.
func (c *ClusterClient) Close() error { func (c *ClusterClient) Close() error {
if !atomic.CompareAndSwapInt32(&c._closed, 0, 1) { c.mu.Lock()
return pool.ErrClosed if !c.closed {
c.closeClients()
c.addrs = nil
c.nodes = nil
c.slots = nil
c.cmdsInfo = nil
} }
c.closed = true
c.clientsMx.Lock() c.mu.Unlock()
c.resetClients()
c.clientsMx.Unlock()
c.setSlots(nil)
return nil return nil
} }
// getClient returns a Client for a given address. func (c *ClusterClient) nodeByAddr(addr string) *clusterNode {
func (c *ClusterClient) getClient(addr string) (*Client, error) { c.mu.RLock()
if c.closed() { node, ok := c.nodes[addr]
return nil, pool.ErrClosed c.mu.RUnlock()
}
if addr == "" {
return c.randomClient()
}
c.clientsMx.RLock()
client, ok := c.clients[addr]
c.clientsMx.RUnlock()
if ok { if ok {
return client, nil return node
} }
c.clientsMx.Lock() c.mu.Lock()
client, ok = c.clients[addr] if !c.closed {
if !ok { node, ok = c.nodes[addr]
opt := c.opt.clientOptions() if !ok {
opt.Addr = addr node = c.newNode(addr)
client = NewClient(opt) c.nodes[addr] = node
c.clients[addr] = client c.addrs = append(c.addrs, node.Addr)
}
} }
c.clientsMx.Unlock() c.mu.Unlock()
return client, nil return node
} }
func (c *ClusterClient) slotAddrs(slot int) []string { func (c *ClusterClient) newNode(addr string) *clusterNode {
c.slotsMx.RLock() opt := c.opt.clientOptions()
addrs := c.slots[slot] opt.Addr = addr
c.slotsMx.RUnlock() return &clusterNode{
return addrs Addr: addr,
} Client: NewClient(opt),
func (c *ClusterClient) slotMasterAddr(slot int) string {
addrs := c.slotAddrs(slot)
if len(addrs) > 0 {
return addrs[0]
} }
return ""
} }
// randomClient returns a Client for the first live node. func (c *ClusterClient) slotNodes(slot int) []*clusterNode {
func (c *ClusterClient) randomClient() (client *Client, err error) { c.mu.RLock()
nodes := c.slots[slot]
c.mu.RUnlock()
return nodes
}
// randomNode returns random live node.
func (c *ClusterClient) randomNode() *clusterNode {
var node *clusterNode
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
n := rand.Intn(len(c.addrs)) c.mu.RLock()
client, err = c.getClient(c.addrs[n]) addrs := c.addrs
if err != nil { c.mu.RUnlock()
continue
if len(addrs) == 0 {
return nil
} }
err = client.ClusterInfo().Err()
if err == nil { n := rand.Intn(len(addrs))
return client, nil node = c.nodeByAddr(addrs[n])
if node.Client.ClusterInfo().Err() == nil {
return node
} }
} }
return nil, err return node
}
func (c *ClusterClient) slotMasterNode(slot int) *clusterNode {
nodes := c.slotNodes(slot)
if len(nodes) == 0 {
return c.randomNode()
}
return nodes[0]
}
func (c *ClusterClient) slotSlaveNode(slot int) *clusterNode {
nodes := c.slotNodes(slot)
switch len(nodes) {
case 0:
return c.randomNode()
case 1:
return nodes[0]
case 2:
return nodes[1]
default:
n := rand.Intn(len(nodes)-1) + 1
return nodes[n]
}
}
func (c *ClusterClient) slotClosestNode(slot int) *clusterNode {
nodes := c.slotNodes(slot)
var node *clusterNode
for _, n := range nodes {
if node == nil || n.Latency < node.Latency {
node = n
}
}
return node
}
func (c *ClusterClient) cmdSlotAndNode(cmd Cmder) (int, *clusterNode) {
cmdInfo := c.cmdInfo(cmd.arg(0))
if cmdInfo == nil {
return 0, c.randomNode()
}
if cmdInfo.FirstKeyPos == -1 {
return 0, c.randomNode()
}
firstKey := cmd.arg(int(cmdInfo.FirstKeyPos))
slot := hashtag.Slot(firstKey)
if cmdInfo.ReadOnly && c.opt.ReadOnly {
if c.opt.RouteByLatency {
return slot, c.slotClosestNode(slot)
}
return slot, c.slotSlaveNode(slot)
}
return slot, c.slotMasterNode(slot)
} }
func (c *ClusterClient) process(cmd Cmder) { func (c *ClusterClient) process(cmd Cmder) {
var ask bool var ask bool
slot, node := c.cmdSlotAndNode(cmd)
slot := hashtag.Slot(cmd.clusterKey())
addr := c.slotMasterAddr(slot)
client, err := c.getClient(addr)
if err != nil {
cmd.setErr(err)
return
}
for attempt := 0; attempt <= c.opt.getMaxRedirects(); attempt++ { for attempt := 0; attempt <= c.opt.getMaxRedirects(); attempt++ {
if attempt > 0 { if attempt > 0 {
cmd.reset() cmd.reset()
} }
if node == nil {
cmd.setErr(pool.ErrClosed)
return
}
if ask { if ask {
pipe := client.Pipeline() pipe := node.Client.Pipeline()
pipe.Process(NewCmd("ASKING")) pipe.Process(NewCmd("ASKING"))
pipe.Process(cmd) pipe.Process(cmd)
_, _ = pipe.Exec() _, _ = pipe.Exec()
pipe.Close() pipe.Close()
ask = false ask = false
} else { } else {
client.Process(cmd) node.Client.Process(cmd)
} }
// If there is no (real) error, we are done! // If there is no (real) error, we are done!
@ -199,10 +274,7 @@ func (c *ClusterClient) process(cmd Cmder) {
// On network errors try random node. // On network errors try random node.
if shouldRetry(err) { if shouldRetry(err) {
client, err = c.randomClient() node = c.randomNode()
if err != nil {
return
}
continue continue
} }
@ -210,13 +282,11 @@ func (c *ClusterClient) process(cmd Cmder) {
var addr string var addr string
moved, ask, addr = isMovedError(err) moved, ask, addr = isMovedError(err)
if moved || ask { if moved || ask {
if moved && c.slotMasterAddr(slot) != addr { if moved && c.slotMasterNode(slot).Addr != addr {
c.lazyReloadSlots() c.lazyReloadSlots()
} }
client, err = c.getClient(addr)
if err != nil { node = c.nodeByAddr(addr)
return
}
continue continue
} }
@ -224,64 +294,71 @@ func (c *ClusterClient) process(cmd Cmder) {
} }
} }
// Closes all clients and returns last error if there are any. // closeClients closes all clients and returns the first error if there are any.
func (c *ClusterClient) resetClients() (retErr error) { func (c *ClusterClient) closeClients() error {
for addr, client := range c.clients { var retErr error
if err := client.Close(); err != nil && retErr == nil { for _, node := range c.nodes {
if err := node.Client.Close(); err != nil && retErr == nil {
retErr = err retErr = err
} }
delete(c.clients, addr)
} }
return retErr return retErr
} }
func (c *ClusterClient) setSlots(slots []ClusterSlot) { func (c *ClusterClient) setSlots(cs []ClusterSlot) {
c.slotsMx.Lock() slots := make([][]*clusterNode, hashtag.SlotNumber)
seen := make(map[string]struct{})
for _, addr := range c.addrs {
seen[addr] = struct{}{}
}
for i := 0; i < hashtag.SlotNumber; i++ { for i := 0; i < hashtag.SlotNumber; i++ {
c.slots[i] = c.slots[i][:0] slots[i] = nil
} }
for _, slot := range slots { for _, s := range cs {
var addrs []string var nodes []*clusterNode
for _, node := range slot.Nodes { for _, n := range s.Nodes {
addrs = append(addrs, node.Addr) nodes = append(nodes, c.nodeByAddr(n.Addr))
} }
for i := slot.Start; i <= slot.End; i++ { for i := s.Start; i <= s.End; i++ {
c.slots[i] = addrs slots[i] = nodes
}
for _, node := range slot.Nodes {
if _, ok := seen[node.Addr]; !ok {
c.addrs = append(c.addrs, node.Addr)
seen[node.Addr] = struct{}{}
}
} }
} }
c.slotsMx.Unlock() c.mu.Lock()
if !c.closed {
c.slots = slots
}
c.mu.Unlock()
}
func (c *ClusterClient) setNodesLatency() {
nodes := c.getNodes()
for _, node := range nodes {
var latency int
for i := 0; i < 10; i++ {
t1 := time.Now()
node.Client.Ping()
latency += int(time.Since(t1) / time.Millisecond)
}
node.Latency = latency
}
} }
func (c *ClusterClient) reloadSlots() { func (c *ClusterClient) reloadSlots() {
defer atomic.StoreUint32(&c.reloading, 0) defer atomic.StoreUint32(&c.reloading, 0)
client, err := c.randomClient() node := c.randomNode()
if err != nil { if node == nil {
internal.Logf("randomClient failed: %s", err)
return return
} }
slots, err := client.ClusterSlots().Result() slots, err := node.Client.ClusterSlots().Result()
if err != nil { if err != nil {
internal.Logf("ClusterSlots failed: %s", err) internal.Logf("ClusterSlots on addr=%q failed: %s", node.Addr, err)
return return
} }
c.setSlots(slots) c.setSlots(slots)
if c.opt.RouteByLatency {
c.setNodesLatency()
}
} }
func (c *ClusterClient) lazyReloadSlots() { func (c *ClusterClient) lazyReloadSlots() {
@ -297,13 +374,14 @@ func (c *ClusterClient) reaper(frequency time.Duration) {
defer ticker.Stop() defer ticker.Stop()
for _ = range ticker.C { for _ = range ticker.C {
if c.closed() { nodes := c.getNodes()
if nodes == nil {
break break
} }
var n int var n int
for _, client := range c.getClients() { for _, node := range nodes {
nn, err := client.connPool.(*pool.ConnPool).ReapStaleConns() nn, err := node.Client.connPool.(*pool.ConnPool).ReapStaleConns()
if err != nil { if err != nil {
internal.Logf("ReapStaleConns failed: %s", err) internal.Logf("ReapStaleConns failed: %s", err)
} else { } else {
@ -334,25 +412,21 @@ func (c *ClusterClient) Pipelined(fn func(*Pipeline) error) ([]Cmder, error) {
func (c *ClusterClient) pipelineExec(cmds []Cmder) error { func (c *ClusterClient) pipelineExec(cmds []Cmder) error {
var retErr error var retErr error
cmdsMap := make(map[string][]Cmder) cmdsMap := make(map[*clusterNode][]Cmder)
for _, cmd := range cmds { for _, cmd := range cmds {
slot := hashtag.Slot(cmd.clusterKey()) _, node := c.cmdSlotAndNode(cmd)
addr := c.slotMasterAddr(slot) cmdsMap[node] = append(cmdsMap[node], cmd)
cmdsMap[addr] = append(cmdsMap[addr], cmd)
} }
for attempt := 0; attempt <= c.opt.getMaxRedirects(); attempt++ { for attempt := 0; attempt <= c.opt.getMaxRedirects(); attempt++ {
failedCmds := make(map[string][]Cmder) failedCmds := make(map[*clusterNode][]Cmder)
for addr, cmds := range cmdsMap { for node, cmds := range cmdsMap {
client, err := c.getClient(addr) if node == nil {
if err != nil { node = c.randomNode()
setCmdsErr(cmds, err)
retErr = err
continue
} }
cn, err := client.conn() cn, err := node.Client.conn()
if err != nil { if err != nil {
setCmdsErr(cmds, err) setCmdsErr(cmds, err)
retErr = err retErr = err
@ -363,7 +437,7 @@ func (c *ClusterClient) pipelineExec(cmds []Cmder) error {
if err != nil { if err != nil {
retErr = err retErr = err
} }
client.putConn(cn, err, false) node.Client.putConn(cn, err, false)
} }
cmdsMap = failedCmds cmdsMap = failedCmds
@ -373,8 +447,8 @@ func (c *ClusterClient) pipelineExec(cmds []Cmder) error {
} }
func (c *ClusterClient) execClusterCmds( func (c *ClusterClient) execClusterCmds(
cn *pool.Conn, cmds []Cmder, failedCmds map[string][]Cmder, cn *pool.Conn, cmds []Cmder, failedCmds map[*clusterNode][]Cmder,
) (map[string][]Cmder, error) { ) (map[*clusterNode][]Cmder, error) {
if err := writeCmd(cn, cmds...); err != nil { if err := writeCmd(cn, cmds...); err != nil {
setCmdsErr(cmds, err) setCmdsErr(cmds, err)
return failedCmds, err return failedCmds, err
@ -388,15 +462,17 @@ func (c *ClusterClient) execClusterCmds(
} }
if isNetworkError(err) { if isNetworkError(err) {
cmd.reset() cmd.reset()
failedCmds[""] = append(failedCmds[""], cmds[i:]...) failedCmds[nil] = append(failedCmds[nil], cmds[i:]...)
break break
} else if moved, ask, addr := isMovedError(err); moved { } else if moved, ask, addr := isMovedError(err); moved {
c.lazyReloadSlots() c.lazyReloadSlots()
cmd.reset() cmd.reset()
failedCmds[addr] = append(failedCmds[addr], cmd) node := c.nodeByAddr(addr)
failedCmds[node] = append(failedCmds[node], cmd)
} else if ask { } else if ask {
cmd.reset() cmd.reset()
failedCmds[addr] = append(failedCmds[addr], NewCmd("ASKING"), cmd) node := c.nodeByAddr(addr)
failedCmds[node] = append(failedCmds[node], NewCmd("ASKING"), cmd)
} else if firstCmdErr == nil { } else if firstCmdErr == nil {
firstCmdErr = err firstCmdErr = err
} }
@ -418,6 +494,12 @@ type ClusterOptions struct {
// Default is 16. // Default is 16.
MaxRedirects int MaxRedirects int
// Enables read queries for a connection to a Redis Cluster slave node.
ReadOnly bool
// Enables routing read-only queries to the closest master or slave node.
RouteByLatency bool
// Following options are copied from Options struct. // Following options are copied from Options struct.
Password string Password string
@ -446,6 +528,7 @@ func (opt *ClusterOptions) getMaxRedirects() int {
func (opt *ClusterOptions) clientOptions() *Options { func (opt *ClusterOptions) clientOptions() *Options {
return &Options{ return &Options{
Password: opt.Password, Password: opt.Password,
ReadOnly: opt.ReadOnly,
DialTimeout: opt.DialTimeout, DialTimeout: opt.DialTimeout,
ReadTimeout: opt.ReadTimeout, ReadTimeout: opt.ReadTimeout,
@ -454,6 +537,7 @@ func (opt *ClusterOptions) clientOptions() *Options {
PoolSize: opt.PoolSize, PoolSize: opt.PoolSize,
PoolTimeout: opt.PoolTimeout, PoolTimeout: opt.PoolTimeout,
IdleTimeout: opt.IdleTimeout, IdleTimeout: opt.IdleTimeout,
// IdleCheckFrequency is not copied to disable reaper // IdleCheckFrequency is not copied to disable reaper
} }
} }

View File

@ -6,16 +6,21 @@ import (
) )
func (c *ClusterClient) SlotAddrs(slot int) []string { func (c *ClusterClient) SlotAddrs(slot int) []string {
return c.slotAddrs(slot) var addrs []string
for _, n := range c.slotNodes(slot) {
addrs = append(addrs, n.Addr)
}
return addrs
} }
// SwapSlot swaps a slot's master/slave address // SwapSlot swaps a slot's master/slave address
// for testing MOVED redirects // for testing MOVED redirects
func (c *ClusterClient) SwapSlot(pos int) []string { func (c *ClusterClient) SwapSlotNodes(slot int) []string {
c.slotsMx.Lock() c.mu.Lock()
defer c.slotsMx.Unlock() nodes := c.slots[slot]
c.slots[pos][0], c.slots[pos][1] = c.slots[pos][1], c.slots[pos][0] nodes[0], nodes[1] = nodes[1], nodes[0]
return c.slots[pos] c.mu.Unlock()
return c.SlotAddrs(slot)
} }
var _ = Describe("ClusterClient", func() { var _ = Describe("ClusterClient", func() {
@ -42,19 +47,26 @@ var _ = Describe("ClusterClient", func() {
It("should initialize", func() { It("should initialize", func() {
Expect(subject.addrs).To(HaveLen(3)) Expect(subject.addrs).To(HaveLen(3))
Expect(subject.slots).To(HaveLen(16384))
}) })
It("should update slots cache", func() { It("should update slots cache", func() {
populate() populate()
Expect(subject.slots[0]).To(Equal([]string{"127.0.0.1:7000", "127.0.0.1:7004"})) Expect(subject.slots[0][0].Addr).To(Equal("127.0.0.1:7000"))
Expect(subject.slots[4095]).To(Equal([]string{"127.0.0.1:7000", "127.0.0.1:7004"})) Expect(subject.slots[0][1].Addr).To(Equal("127.0.0.1:7004"))
Expect(subject.slots[4096]).To(Equal([]string{"127.0.0.1:7001", "127.0.0.1:7005"})) Expect(subject.slots[4095][0].Addr).To(Equal("127.0.0.1:7000"))
Expect(subject.slots[8191]).To(Equal([]string{"127.0.0.1:7001", "127.0.0.1:7005"})) Expect(subject.slots[4095][1].Addr).To(Equal("127.0.0.1:7004"))
Expect(subject.slots[8192]).To(Equal([]string{"127.0.0.1:7002", "127.0.0.1:7006"})) Expect(subject.slots[4096][0].Addr).To(Equal("127.0.0.1:7001"))
Expect(subject.slots[12287]).To(Equal([]string{"127.0.0.1:7002", "127.0.0.1:7006"})) Expect(subject.slots[4096][1].Addr).To(Equal("127.0.0.1:7005"))
Expect(subject.slots[12288]).To(Equal([]string{"127.0.0.1:7003", "127.0.0.1:7007"})) Expect(subject.slots[8191][0].Addr).To(Equal("127.0.0.1:7001"))
Expect(subject.slots[16383]).To(Equal([]string{"127.0.0.1:7003", "127.0.0.1:7007"})) Expect(subject.slots[8191][1].Addr).To(Equal("127.0.0.1:7005"))
Expect(subject.slots[8192][0].Addr).To(Equal("127.0.0.1:7002"))
Expect(subject.slots[8192][1].Addr).To(Equal("127.0.0.1:7006"))
Expect(subject.slots[12287][0].Addr).To(Equal("127.0.0.1:7002"))
Expect(subject.slots[12287][1].Addr).To(Equal("127.0.0.1:7006"))
Expect(subject.slots[12288][0].Addr).To(Equal("127.0.0.1:7003"))
Expect(subject.slots[12288][1].Addr).To(Equal("127.0.0.1:7007"))
Expect(subject.slots[16383][0].Addr).To(Equal("127.0.0.1:7003"))
Expect(subject.slots[16383][1].Addr).To(Equal("127.0.0.1:7007"))
Expect(subject.addrs).To(Equal([]string{ Expect(subject.addrs).To(Equal([]string{
"127.0.0.1:6379", "127.0.0.1:6379",
"127.0.0.1:7003", "127.0.0.1:7003",
@ -71,11 +83,9 @@ var _ = Describe("ClusterClient", func() {
It("should close", func() { It("should close", func() {
populate() populate()
Expect(subject.Close()).NotTo(HaveOccurred()) Expect(subject.Close()).NotTo(HaveOccurred())
Expect(subject.clients).To(BeEmpty()) Expect(subject.addrs).To(BeEmpty())
Expect(subject.slots[0]).To(BeEmpty()) Expect(subject.nodes).To(BeEmpty())
Expect(subject.slots[8191]).To(BeEmpty()) Expect(subject.slots).To(BeEmpty())
Expect(subject.slots[8192]).To(BeEmpty())
Expect(subject.slots[16383]).To(BeEmpty())
Expect(subject.Ping().Err().Error()).To(Equal("redis: client is closed")) Expect(subject.Ping().Err().Error()).To(Equal("redis: client is closed"))
}) })
}) })

View File

@ -301,7 +301,7 @@ var _ = Describe("Cluster", func() {
}) })
It("should CLUSTER READONLY", func() { It("should CLUSTER READONLY", func() {
res, err := cluster.primary().Readonly().Result() res, err := cluster.primary().ReadOnly().Result()
Expect(err).NotTo(HaveOccurred()) Expect(err).NotTo(HaveOccurred())
Expect(res).To(Equal("OK")) Expect(res).To(Equal("OK"))
}) })
@ -353,7 +353,7 @@ var _ = Describe("Cluster", func() {
Expect(client.Set("A", "VALUE", 0).Err()).NotTo(HaveOccurred()) Expect(client.Set("A", "VALUE", 0).Err()).NotTo(HaveOccurred())
slot := hashtag.Slot("A") slot := hashtag.Slot("A")
Expect(client.SwapSlot(slot)).To(Equal([]string{"127.0.0.1:8224", "127.0.0.1:8221"})) Expect(client.SwapSlotNodes(slot)).To(Equal([]string{"127.0.0.1:8224", "127.0.0.1:8221"}))
val, err := client.Get("A").Result() val, err := client.Get("A").Result()
Expect(err).NotTo(HaveOccurred()) Expect(err).NotTo(HaveOccurred())
@ -361,7 +361,7 @@ var _ = Describe("Cluster", func() {
Eventually(func() []string { Eventually(func() []string {
return client.SlotAddrs(slot) return client.SlotAddrs(slot)
}, "5s").Should(Equal([]string{"127.0.0.1:8221", "127.0.0.1:8224"})) }, "10s").Should(Equal([]string{"127.0.0.1:8221", "127.0.0.1:8224"}))
}) })
It("should return error when there are no attempts left", func() { It("should return error when there are no attempts left", func() {
@ -371,7 +371,7 @@ var _ = Describe("Cluster", func() {
}) })
slot := hashtag.Slot("A") slot := hashtag.Slot("A")
Expect(client.SwapSlot(slot)).To(Equal([]string{"127.0.0.1:8224", "127.0.0.1:8221"})) Expect(client.SwapSlotNodes(slot)).To(Equal([]string{"127.0.0.1:8224", "127.0.0.1:8221"}))
err := client.Get("A").Err() err := client.Get("A").Err()
Expect(err).To(HaveOccurred()) Expect(err).To(HaveOccurred())
@ -435,7 +435,7 @@ var _ = Describe("Cluster", func() {
It("performs multi-pipelines", func() { It("performs multi-pipelines", func() {
slot := hashtag.Slot("A") slot := hashtag.Slot("A")
Expect(client.SwapSlot(slot)).To(Equal([]string{"127.0.0.1:8224", "127.0.0.1:8221"})) Expect(client.SwapSlotNodes(slot)).To(Equal([]string{"127.0.0.1:8224", "127.0.0.1:8221"}))
pipe := client.Pipeline() pipe := client.Pipeline()
defer pipe.Close() defer pipe.Close()

View File

@ -30,12 +30,12 @@ var (
type Cmder interface { type Cmder interface {
args() []interface{} args() []interface{}
arg(int) string
readReply(*pool.Conn) error readReply(*pool.Conn) error
setErr(error) setErr(error)
reset() reset()
readTimeout() *time.Duration readTimeout() *time.Duration
clusterKey() string
Err() error Err() error
fmt.Stringer fmt.Stringer
@ -92,10 +92,7 @@ func cmdString(cmd Cmder, val interface{}) string {
type baseCmd struct { type baseCmd struct {
_args []interface{} _args []interface{}
err error
err error
_clusterKeyPos int
_readTimeout *time.Duration _readTimeout *time.Duration
} }
@ -111,6 +108,15 @@ func (cmd *baseCmd) args() []interface{} {
return cmd._args return cmd._args
} }
func (cmd *baseCmd) arg(pos int) string {
if len(cmd._args) > pos {
if s, ok := cmd._args[pos].(string); ok {
return s
}
}
return ""
}
func (cmd *baseCmd) readTimeout() *time.Duration { func (cmd *baseCmd) readTimeout() *time.Duration {
return cmd._readTimeout return cmd._readTimeout
} }
@ -119,17 +125,14 @@ func (cmd *baseCmd) setReadTimeout(d time.Duration) {
cmd._readTimeout = &d cmd._readTimeout = &d
} }
func (cmd *baseCmd) clusterKey() string {
if cmd._clusterKeyPos > 0 && cmd._clusterKeyPos < len(cmd._args) {
return fmt.Sprint(cmd._args[cmd._clusterKeyPos])
}
return ""
}
func (cmd *baseCmd) setErr(e error) { func (cmd *baseCmd) setErr(e error) {
cmd.err = e cmd.err = e
} }
func newBaseCmd(args []interface{}) baseCmd {
return baseCmd{_args: args}
}
//------------------------------------------------------------------------------ //------------------------------------------------------------------------------
type Cmd struct { type Cmd struct {
@ -139,7 +142,7 @@ type Cmd struct {
} }
func NewCmd(args ...interface{}) *Cmd { func NewCmd(args ...interface{}) *Cmd {
return &Cmd{baseCmd: baseCmd{_args: args}} return &Cmd{baseCmd: newBaseCmd(args)}
} }
func (cmd *Cmd) reset() { func (cmd *Cmd) reset() {
@ -183,7 +186,8 @@ type SliceCmd struct {
} }
func NewSliceCmd(args ...interface{}) *SliceCmd { func NewSliceCmd(args ...interface{}) *SliceCmd {
return &SliceCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &SliceCmd{baseCmd: cmd}
} }
func (cmd *SliceCmd) reset() { func (cmd *SliceCmd) reset() {
@ -222,11 +226,8 @@ type StatusCmd struct {
} }
func NewStatusCmd(args ...interface{}) *StatusCmd { func NewStatusCmd(args ...interface{}) *StatusCmd {
return &StatusCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
} return &StatusCmd{baseCmd: cmd}
func newKeylessStatusCmd(args ...interface{}) *StatusCmd {
return &StatusCmd{baseCmd: baseCmd{_args: args}}
} }
func (cmd *StatusCmd) reset() { func (cmd *StatusCmd) reset() {
@ -260,7 +261,8 @@ type IntCmd struct {
} }
func NewIntCmd(args ...interface{}) *IntCmd { func NewIntCmd(args ...interface{}) *IntCmd {
return &IntCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &IntCmd{baseCmd: cmd}
} }
func (cmd *IntCmd) reset() { func (cmd *IntCmd) reset() {
@ -295,9 +297,10 @@ type DurationCmd struct {
} }
func NewDurationCmd(precision time.Duration, args ...interface{}) *DurationCmd { func NewDurationCmd(precision time.Duration, args ...interface{}) *DurationCmd {
cmd := newBaseCmd(args)
return &DurationCmd{ return &DurationCmd{
precision: precision, precision: precision,
baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}, baseCmd: cmd,
} }
} }
@ -337,7 +340,8 @@ type BoolCmd struct {
} }
func NewBoolCmd(args ...interface{}) *BoolCmd { func NewBoolCmd(args ...interface{}) *BoolCmd {
return &BoolCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &BoolCmd{baseCmd: cmd}
} }
func (cmd *BoolCmd) reset() { func (cmd *BoolCmd) reset() {
@ -393,7 +397,8 @@ type StringCmd struct {
} }
func NewStringCmd(args ...interface{}) *StringCmd { func NewStringCmd(args ...interface{}) *StringCmd {
return &StringCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &StringCmd{baseCmd: cmd}
} }
func (cmd *StringCmd) reset() { func (cmd *StringCmd) reset() {
@ -468,7 +473,8 @@ type FloatCmd struct {
} }
func NewFloatCmd(args ...interface{}) *FloatCmd { func NewFloatCmd(args ...interface{}) *FloatCmd {
return &FloatCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &FloatCmd{baseCmd: cmd}
} }
func (cmd *FloatCmd) reset() { func (cmd *FloatCmd) reset() {
@ -502,7 +508,8 @@ type StringSliceCmd struct {
} }
func NewStringSliceCmd(args ...interface{}) *StringSliceCmd { func NewStringSliceCmd(args ...interface{}) *StringSliceCmd {
return &StringSliceCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &StringSliceCmd{baseCmd: cmd}
} }
func (cmd *StringSliceCmd) reset() { func (cmd *StringSliceCmd) reset() {
@ -541,7 +548,8 @@ type BoolSliceCmd struct {
} }
func NewBoolSliceCmd(args ...interface{}) *BoolSliceCmd { func NewBoolSliceCmd(args ...interface{}) *BoolSliceCmd {
return &BoolSliceCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &BoolSliceCmd{baseCmd: cmd}
} }
func (cmd *BoolSliceCmd) reset() { func (cmd *BoolSliceCmd) reset() {
@ -580,7 +588,8 @@ type StringStringMapCmd struct {
} }
func NewStringStringMapCmd(args ...interface{}) *StringStringMapCmd { func NewStringStringMapCmd(args ...interface{}) *StringStringMapCmd {
return &StringStringMapCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &StringStringMapCmd{baseCmd: cmd}
} }
func (cmd *StringStringMapCmd) reset() { func (cmd *StringStringMapCmd) reset() {
@ -619,7 +628,8 @@ type StringIntMapCmd struct {
} }
func NewStringIntMapCmd(args ...interface{}) *StringIntMapCmd { func NewStringIntMapCmd(args ...interface{}) *StringIntMapCmd {
return &StringIntMapCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &StringIntMapCmd{baseCmd: cmd}
} }
func (cmd *StringIntMapCmd) Val() map[string]int64 { func (cmd *StringIntMapCmd) Val() map[string]int64 {
@ -658,7 +668,8 @@ type ZSliceCmd struct {
} }
func NewZSliceCmd(args ...interface{}) *ZSliceCmd { func NewZSliceCmd(args ...interface{}) *ZSliceCmd {
return &ZSliceCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &ZSliceCmd{baseCmd: cmd}
} }
func (cmd *ZSliceCmd) reset() { func (cmd *ZSliceCmd) reset() {
@ -698,8 +709,9 @@ type ScanCmd struct {
} }
func NewScanCmd(args ...interface{}) *ScanCmd { func NewScanCmd(args ...interface{}) *ScanCmd {
cmd := newBaseCmd(args)
return &ScanCmd{ return &ScanCmd{
baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}, baseCmd: cmd,
} }
} }
@ -752,7 +764,8 @@ type ClusterSlotsCmd struct {
} }
func NewClusterSlotsCmd(args ...interface{}) *ClusterSlotsCmd { func NewClusterSlotsCmd(args ...interface{}) *ClusterSlotsCmd {
return &ClusterSlotsCmd{baseCmd: baseCmd{_args: args, _clusterKeyPos: 1}} cmd := newBaseCmd(args)
return &ClusterSlotsCmd{baseCmd: cmd}
} }
func (cmd *ClusterSlotsCmd) Val() []ClusterSlot { func (cmd *ClusterSlotsCmd) Val() []ClusterSlot {
@ -833,12 +846,10 @@ func NewGeoLocationCmd(q *GeoRadiusQuery, args ...interface{}) *GeoLocationCmd {
if q.Sort != "" { if q.Sort != "" {
args = append(args, q.Sort) args = append(args, q.Sort)
} }
cmd := newBaseCmd(args)
return &GeoLocationCmd{ return &GeoLocationCmd{
baseCmd: baseCmd{ baseCmd: cmd,
_args: args, q: q,
_clusterKeyPos: 1,
},
q: q,
} }
} }
@ -868,3 +879,53 @@ func (cmd *GeoLocationCmd) readReply(cn *pool.Conn) error {
cmd.locations = reply.([]GeoLocation) cmd.locations = reply.([]GeoLocation)
return nil return nil
} }
//------------------------------------------------------------------------------
type CommandInfo struct {
Name string
Arity int8
Flags []string
FirstKeyPos int8
LastKeyPos int8
StepCount int8
ReadOnly bool
}
type CommandsInfoCmd struct {
baseCmd
val map[string]*CommandInfo
}
func NewCommandsInfoCmd(args ...interface{}) *CommandsInfoCmd {
cmd := newBaseCmd(args)
return &CommandsInfoCmd{baseCmd: cmd}
}
func (cmd *CommandsInfoCmd) Val() map[string]*CommandInfo {
return cmd.val
}
func (cmd *CommandsInfoCmd) Result() (map[string]*CommandInfo, error) {
return cmd.Val(), cmd.Err()
}
func (cmd *CommandsInfoCmd) String() string {
return cmdString(cmd, cmd.val)
}
func (cmd *CommandsInfoCmd) reset() {
cmd.val = nil
cmd.err = nil
}
func (cmd *CommandsInfoCmd) readReply(cn *pool.Conn) error {
v, err := readArrayReply(cn, commandInfoSliceParser)
if err != nil {
cmd.err = err
return err
}
cmd.val = v.(map[string]*CommandInfo)
return nil
}

View File

@ -21,10 +21,10 @@ var _ = Describe("Cmd", func() {
It("implements Stringer", func() { It("implements Stringer", func() {
set := client.Set("foo", "bar", 0) set := client.Set("foo", "bar", 0)
Expect(set.String()).To(Equal("SET foo bar: OK")) Expect(set.String()).To(Equal("set foo bar: OK"))
get := client.Get("foo") get := client.Get("foo")
Expect(get.String()).To(Equal("GET foo: bar")) Expect(get.String()).To(Equal("get foo: bar"))
}) })
It("has val/err", func() { It("has val/err", func() {

File diff suppressed because it is too large Load Diff

View File

@ -2729,6 +2729,7 @@ var _ = Describe("Commands", func() {
}) })
Describe("json marshaling/unmarshaling", func() { Describe("json marshaling/unmarshaling", func() {
BeforeEach(func() { BeforeEach(func() {
value := &numberStruct{Number: 42} value := &numberStruct{Number: 42}
err := client.Set("key", value, 0).Err() err := client.Set("key", value, 0).Err()
@ -2744,12 +2745,30 @@ var _ = Describe("Commands", func() {
It("should scan custom values using json", func() { It("should scan custom values using json", func() {
value := &numberStruct{} value := &numberStruct{}
err := client.Get("key").Scan(value) err := client.Get("key").Scan(value)
Expect(err).To(BeNil()) Expect(err).NotTo(HaveOccurred())
Expect(value.Number).To(Equal(42)) Expect(value.Number).To(Equal(42))
}) })
}) })
Describe("Command", func() {
It("returns map of commands", func() {
cmds, err := client.Command().Result()
Expect(err).NotTo(HaveOccurred())
Expect(len(cmds)).To(BeNumerically("~", 173, 5))
cmd := cmds["mget"]
Expect(cmd.Name).To(Equal("mget"))
Expect(cmd.Arity).To(Equal(int8(-2)))
Expect(cmd.Flags).To(Equal([]string{"readonly"}))
Expect(cmd.FirstKeyPos).To(Equal(int8(1)))
Expect(cmd.LastKeyPos).To(Equal(int8(-1)))
Expect(cmd.StepCount).To(Equal(int8(1)))
})
})
}) })
type numberStruct struct { type numberStruct struct {

View File

@ -53,6 +53,9 @@ type Options struct {
// The frequency of idle checks. // The frequency of idle checks.
// Default is 1 minute. // Default is 1 minute.
IdleCheckFrequency time.Duration IdleCheckFrequency time.Duration
// Enables read queries for a connection to a Redis Cluster slave node.
ReadOnly bool
} }
func (opt *Options) getNetwork() string { func (opt *Options) getNetwork() string {

View File

@ -627,8 +627,8 @@ func clusterSlotsParser(cn *pool.Conn, slotNum int64) (interface{}, error) {
func newGeoLocationParser(q *GeoRadiusQuery) multiBulkParser { func newGeoLocationParser(q *GeoRadiusQuery) multiBulkParser {
return func(cn *pool.Conn, n int64) (interface{}, error) { return func(cn *pool.Conn, n int64) (interface{}, error) {
var loc GeoLocation var loc GeoLocation
var err error var err error
loc.Name, err = readStringReply(cn) loc.Name, err = readStringReply(cn)
if err != nil { if err != nil {
return nil, err return nil, err
@ -690,3 +690,70 @@ func newGeoLocationSliceParser(q *GeoRadiusQuery) multiBulkParser {
return locs, nil return locs, nil
} }
} }
func commandInfoParser(cn *pool.Conn, n int64) (interface{}, error) {
var cmd CommandInfo
var err error
if n != 6 {
return nil, fmt.Errorf("redis: got %d elements in COMMAND reply, wanted 6")
}
cmd.Name, err = readStringReply(cn)
if err != nil {
return nil, err
}
arity, err := readIntReply(cn)
if err != nil {
return nil, err
}
cmd.Arity = int8(arity)
flags, err := readReply(cn, stringSliceParser)
if err != nil {
return nil, err
}
cmd.Flags = flags.([]string)
firstKeyPos, err := readIntReply(cn)
if err != nil {
return nil, err
}
cmd.FirstKeyPos = int8(firstKeyPos)
lastKeyPos, err := readIntReply(cn)
if err != nil {
return nil, err
}
cmd.LastKeyPos = int8(lastKeyPos)
stepCount, err := readIntReply(cn)
if err != nil {
return nil, err
}
cmd.StepCount = int8(stepCount)
for _, flag := range cmd.Flags {
if flag == "readonly" {
cmd.ReadOnly = true
break
}
}
return &cmd, nil
}
func commandInfoSliceParser(cn *pool.Conn, n int64) (interface{}, error) {
m := make(map[string]*CommandInfo, n)
for i := int64(0); i < n; i++ {
v, err := readReply(cn, commandInfoParser)
if err != nil {
return nil, err
}
vv := v.(*CommandInfo)
m[vv.Name] = vv
}
return m, nil
}

View File

@ -52,7 +52,7 @@ func (c *baseClient) putConn(cn *pool.Conn, err error, allowTimeout bool) bool {
func (c *baseClient) initConn(cn *pool.Conn) error { func (c *baseClient) initConn(cn *pool.Conn) error {
cn.Inited = true cn.Inited = true
if c.opt.Password == "" && c.opt.DB == 0 { if c.opt.Password == "" && c.opt.DB == 0 && !c.opt.ReadOnly {
return nil return nil
} }
@ -71,6 +71,12 @@ func (c *baseClient) initConn(cn *pool.Conn) error {
} }
} }
if c.opt.ReadOnly {
if err := client.ReadOnly().Err(); err != nil {
return err
}
}
return nil return nil
} }

60
ring.go
View File

@ -115,10 +115,13 @@ type Ring struct {
opt *RingOptions opt *RingOptions
nreplicas int nreplicas int
mx sync.RWMutex mu sync.RWMutex
hash *consistenthash.Map hash *consistenthash.Map
shards map[string]*ringShard shards map[string]*ringShard
cmdsInfo map[string]*CommandInfo
cmdsInfoOnce *sync.Once
closed bool closed bool
} }
@ -130,6 +133,8 @@ func NewRing(opt *RingOptions) *Ring {
hash: consistenthash.New(nreplicas, nil), hash: consistenthash.New(nreplicas, nil),
shards: make(map[string]*ringShard), shards: make(map[string]*ringShard),
cmdsInfoOnce: new(sync.Once),
} }
ring.commandable.process = ring.process ring.commandable.process = ring.process
for name, addr := range opt.Addrs { for name, addr := range opt.Addrs {
@ -141,15 +146,40 @@ func NewRing(opt *RingOptions) *Ring {
return ring return ring
} }
func (ring *Ring) cmdInfo(name string) *CommandInfo {
ring.cmdsInfoOnce.Do(func() {
for _, shard := range ring.shards {
cmdsInfo, err := shard.Client.Command().Result()
if err == nil {
ring.cmdsInfo = cmdsInfo
return
}
}
ring.cmdsInfoOnce = &sync.Once{}
})
if ring.cmdsInfo == nil {
return nil
}
return ring.cmdsInfo[name]
}
func (ring *Ring) cmdFirstKey(cmd Cmder) string {
cmdInfo := ring.cmdInfo(cmd.arg(0))
if cmdInfo == nil {
return ""
}
return cmd.arg(int(cmdInfo.FirstKeyPos))
}
func (ring *Ring) addClient(name string, cl *Client) { func (ring *Ring) addClient(name string, cl *Client) {
ring.mx.Lock() ring.mu.Lock()
ring.hash.Add(name) ring.hash.Add(name)
ring.shards[name] = &ringShard{Client: cl} ring.shards[name] = &ringShard{Client: cl}
ring.mx.Unlock() ring.mu.Unlock()
} }
func (ring *Ring) getClient(key string) (*Client, error) { func (ring *Ring) getClient(key string) (*Client, error) {
ring.mx.RLock() ring.mu.RLock()
if ring.closed { if ring.closed {
return nil, pool.ErrClosed return nil, pool.ErrClosed
@ -157,17 +187,17 @@ func (ring *Ring) getClient(key string) (*Client, error) {
name := ring.hash.Get(hashtag.Key(key)) name := ring.hash.Get(hashtag.Key(key))
if name == "" { if name == "" {
ring.mx.RUnlock() ring.mu.RUnlock()
return nil, errRingShardsDown return nil, errRingShardsDown
} }
cl := ring.shards[name].Client cl := ring.shards[name].Client
ring.mx.RUnlock() ring.mu.RUnlock()
return cl, nil return cl, nil
} }
func (ring *Ring) process(cmd Cmder) { func (ring *Ring) process(cmd Cmder) {
cl, err := ring.getClient(cmd.clusterKey()) cl, err := ring.getClient(ring.cmdFirstKey(cmd))
if err != nil { if err != nil {
cmd.setErr(err) cmd.setErr(err)
return return
@ -177,8 +207,8 @@ func (ring *Ring) process(cmd Cmder) {
// rebalance removes dead shards from the ring. // rebalance removes dead shards from the ring.
func (ring *Ring) rebalance() { func (ring *Ring) rebalance() {
defer ring.mx.Unlock() defer ring.mu.Unlock()
ring.mx.Lock() ring.mu.Lock()
ring.hash = consistenthash.New(ring.nreplicas, nil) ring.hash = consistenthash.New(ring.nreplicas, nil)
for name, shard := range ring.shards { for name, shard := range ring.shards {
@ -195,10 +225,10 @@ func (ring *Ring) heartbeat() {
for _ = range ticker.C { for _ = range ticker.C {
var rebalance bool var rebalance bool
ring.mx.RLock() ring.mu.RLock()
if ring.closed { if ring.closed {
ring.mx.RUnlock() ring.mu.RUnlock()
break break
} }
@ -210,7 +240,7 @@ func (ring *Ring) heartbeat() {
} }
} }
ring.mx.RUnlock() ring.mu.RUnlock()
if rebalance { if rebalance {
ring.rebalance() ring.rebalance()
@ -223,8 +253,8 @@ func (ring *Ring) heartbeat() {
// It is rare to Close a Ring, as the Ring is meant to be long-lived // It is rare to Close a Ring, as the Ring is meant to be long-lived
// and shared between many goroutines. // and shared between many goroutines.
func (ring *Ring) Close() (retErr error) { func (ring *Ring) Close() (retErr error) {
defer ring.mx.Unlock() defer ring.mu.Unlock()
ring.mx.Lock() ring.mu.Lock()
if ring.closed { if ring.closed {
return nil return nil
@ -259,7 +289,7 @@ func (ring *Ring) pipelineExec(cmds []Cmder) error {
cmdsMap := make(map[string][]Cmder) cmdsMap := make(map[string][]Cmder)
for _, cmd := range cmds { for _, cmd := range cmds {
name := ring.hash.Get(hashtag.Key(cmd.clusterKey())) name := ring.hash.Get(hashtag.Key(ring.cmdFirstKey(cmd)))
if name == "" { if name == "" {
cmd.setErr(errRingShardsDown) cmd.setErr(errRingShardsDown)
if retErr == nil { if retErr == nil {