mirror of
https://github.com/redis/go-redis.git
synced 2025-04-17 20:17:02 +03:00
chore: cleanup
This commit is contained in:
parent
6f7f800107
commit
59798f9dba
@ -94,10 +94,10 @@ func GetSlavesAddrByName(ctx context.Context, c *SentinelClient, name string) []
|
|||||||
return parseReplicaAddrs(addrs, false)
|
return parseReplicaAddrs(addrs, false)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *Ring) GetAddr(addr string) *ringShard {
|
func (c *Ring) ShardByName(name string) *ringShard {
|
||||||
return c.shards.GetAddr(addr)
|
return c.sharding.ShardByName(name)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ringShards) GetAddr(addr string) *ringShard {
|
func (c *ringSharding) ShardByName(name string) *ringShard {
|
||||||
return c.shards[addr]
|
return c.shards.m[name]
|
||||||
}
|
}
|
||||||
|
224
ring.go
224
ring.go
@ -48,8 +48,8 @@ type RingOptions struct {
|
|||||||
// Map of name => host:port addresses of ring shards.
|
// Map of name => host:port addresses of ring shards.
|
||||||
Addrs map[string]string
|
Addrs map[string]string
|
||||||
|
|
||||||
// NewClient creates a shard client with provided name and options.
|
// NewClient creates a shard client with provided options.
|
||||||
NewClient func(name string, opt *Options) *Client
|
NewClient func(opt *Options) *Client
|
||||||
|
|
||||||
// Frequency of PING commands sent to check shards availability.
|
// Frequency of PING commands sent to check shards availability.
|
||||||
// Shard is considered down after 3 subsequent failed checks.
|
// Shard is considered down after 3 subsequent failed checks.
|
||||||
@ -95,7 +95,7 @@ type RingOptions struct {
|
|||||||
|
|
||||||
func (opt *RingOptions) init() {
|
func (opt *RingOptions) init() {
|
||||||
if opt.NewClient == nil {
|
if opt.NewClient == nil {
|
||||||
opt.NewClient = func(name string, opt *Options) *Client {
|
opt.NewClient = func(opt *Options) *Client {
|
||||||
return NewClient(opt)
|
return NewClient(opt)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -163,12 +163,12 @@ type ringShard struct {
|
|||||||
addr string
|
addr string
|
||||||
}
|
}
|
||||||
|
|
||||||
func newRingShard(opt *RingOptions, name, addr string) *ringShard {
|
func newRingShard(opt *RingOptions, addr string) *ringShard {
|
||||||
clopt := opt.clientOptions()
|
clopt := opt.clientOptions()
|
||||||
clopt.Addr = addr
|
clopt.Addr = addr
|
||||||
|
|
||||||
return &ringShard{
|
return &ringShard{
|
||||||
Client: opt.NewClient(name, clopt),
|
Client: opt.NewClient(clopt),
|
||||||
addr: addr,
|
addr: addr,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -210,20 +210,23 @@ func (shard *ringShard) Vote(up bool) bool {
|
|||||||
|
|
||||||
//------------------------------------------------------------------------------
|
//------------------------------------------------------------------------------
|
||||||
|
|
||||||
type ringShards struct {
|
type ringSharding struct {
|
||||||
opt *RingOptions
|
opt *RingOptions
|
||||||
|
|
||||||
mu sync.RWMutex
|
mu sync.RWMutex
|
||||||
muClose sync.Mutex
|
shards *ringShards
|
||||||
hash ConsistentHash
|
|
||||||
shards map[string]*ringShard // read only, updated by SetAddrs
|
|
||||||
list []*ringShard // read only, updated by SetAddrs
|
|
||||||
numShard int
|
|
||||||
closed bool
|
closed bool
|
||||||
|
hash ConsistentHash
|
||||||
|
numShard int
|
||||||
}
|
}
|
||||||
|
|
||||||
func newRingShards(opt *RingOptions) *ringShards {
|
type ringShards struct {
|
||||||
c := &ringShards{
|
m map[string]*ringShard
|
||||||
|
list []*ringShard
|
||||||
|
}
|
||||||
|
|
||||||
|
func newRingSharding(opt *RingOptions) *ringSharding {
|
||||||
|
c := &ringSharding{
|
||||||
opt: opt,
|
opt: opt,
|
||||||
}
|
}
|
||||||
c.SetAddrs(opt.Addrs)
|
c.SetAddrs(opt.Addrs)
|
||||||
@ -234,63 +237,75 @@ func newRingShards(opt *RingOptions) *ringShards {
|
|||||||
// SetAddrs replaces the shards in use, such that you can increase and
|
// SetAddrs replaces the shards in use, such that you can increase and
|
||||||
// decrease number of shards, that you use. It will reuse shards that
|
// decrease number of shards, that you use. It will reuse shards that
|
||||||
// existed before and close the ones that will not be used anymore.
|
// existed before and close the ones that will not be used anymore.
|
||||||
func (c *ringShards) SetAddrs(addrs map[string]string) {
|
func (c *ringSharding) SetAddrs(addrs map[string]string) {
|
||||||
c.muClose.Lock()
|
c.mu.Lock()
|
||||||
defer c.muClose.Unlock()
|
|
||||||
if c.closed {
|
if c.closed {
|
||||||
|
c.mu.Unlock()
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
shards := make(map[string]*ringShard)
|
shards, cleanup := newRingShards(c.opt, addrs, c.shards)
|
||||||
unusedShards := make(map[string]*ringShard)
|
|
||||||
|
|
||||||
for k, shard := range c.shards {
|
|
||||||
if addr, ok := addrs[k]; ok && shard.addr == addr {
|
|
||||||
shards[k] = shard
|
|
||||||
} else {
|
|
||||||
unusedShards[k] = shard
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
for k, addr := range addrs {
|
|
||||||
if shard, ok := c.shards[k]; !ok || shard.addr != addr {
|
|
||||||
shards[k] = newRingShard(c.opt, k, addr)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
list := make([]*ringShard, 0, len(shards))
|
|
||||||
for _, shard := range shards {
|
|
||||||
list = append(list, shard)
|
|
||||||
}
|
|
||||||
|
|
||||||
c.mu.Lock()
|
|
||||||
c.shards = shards
|
c.shards = shards
|
||||||
c.list = list
|
|
||||||
|
|
||||||
c.rebalanceLocked()
|
|
||||||
c.mu.Unlock()
|
c.mu.Unlock()
|
||||||
|
|
||||||
for k, shard := range unusedShards {
|
c.rebalance()
|
||||||
err := shard.Client.Close()
|
cleanup()
|
||||||
if err != nil {
|
}
|
||||||
internal.Logger.Printf(context.Background(), "Failed to close ring shard client %s %s: %v", k, shard.addr, err)
|
|
||||||
|
func newRingShards(
|
||||||
|
opt *RingOptions, addrs map[string]string, existingShards *ringShards,
|
||||||
|
) (*ringShards, func()) {
|
||||||
|
shardMap := make(map[string]*ringShard) // indexed by addr
|
||||||
|
unusedShards := make(map[string]*ringShard) // indexed by addr
|
||||||
|
|
||||||
|
if existingShards != nil {
|
||||||
|
for _, shard := range existingShards.list {
|
||||||
|
addr := shard.Client.opt.Addr
|
||||||
|
shardMap[addr] = shard
|
||||||
|
unusedShards[addr] = shard
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
shards := &ringShards{
|
||||||
|
m: make(map[string]*ringShard),
|
||||||
|
}
|
||||||
|
|
||||||
|
for name, addr := range addrs {
|
||||||
|
if shard, ok := shardMap[addr]; ok {
|
||||||
|
shards.m[name] = shard
|
||||||
|
delete(unusedShards, addr)
|
||||||
|
} else {
|
||||||
|
shards.m[name] = newRingShard(opt, addr)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, shard := range shards.m {
|
||||||
|
shards.list = append(shards.list, shard)
|
||||||
|
}
|
||||||
|
|
||||||
|
return shards, func() {
|
||||||
|
for addr, shard := range unusedShards {
|
||||||
|
if err := shard.Client.Close(); err != nil {
|
||||||
|
internal.Logger.Printf(context.Background(), "shard.Close %s failed: %s", addr, err)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ringShards) List() []*ringShard {
|
func (c *ringSharding) List() []*ringShard {
|
||||||
var list []*ringShard
|
var list []*ringShard
|
||||||
|
|
||||||
c.mu.RLock()
|
c.mu.RLock()
|
||||||
if !c.closed {
|
if !c.closed {
|
||||||
list = c.list
|
list = c.shards.list
|
||||||
}
|
}
|
||||||
c.mu.RUnlock()
|
c.mu.RUnlock()
|
||||||
|
|
||||||
return list
|
return list
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ringShards) Hash(key string) string {
|
func (c *ringSharding) Hash(key string) string {
|
||||||
key = hashtag.Key(key)
|
key = hashtag.Key(key)
|
||||||
|
|
||||||
var hash string
|
var hash string
|
||||||
@ -305,7 +320,7 @@ func (c *ringShards) Hash(key string) string {
|
|||||||
return hash
|
return hash
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ringShards) GetByKey(key string) (*ringShard, error) {
|
func (c *ringSharding) GetByKey(key string) (*ringShard, error) {
|
||||||
key = hashtag.Key(key)
|
key = hashtag.Key(key)
|
||||||
|
|
||||||
c.mu.RLock()
|
c.mu.RLock()
|
||||||
@ -319,15 +334,14 @@ func (c *ringShards) GetByKey(key string) (*ringShard, error) {
|
|||||||
return nil, errRingShardsDown
|
return nil, errRingShardsDown
|
||||||
}
|
}
|
||||||
|
|
||||||
hash := c.hash.Get(key)
|
shardName := c.hash.Get(key)
|
||||||
if hash == "" {
|
if shardName == "" {
|
||||||
return nil, errRingShardsDown
|
return nil, errRingShardsDown
|
||||||
}
|
}
|
||||||
|
return c.shards.m[shardName], nil
|
||||||
return c.shards[hash], nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ringShards) GetByName(shardName string) (*ringShard, error) {
|
func (c *ringSharding) GetByName(shardName string) (*ringShard, error) {
|
||||||
if shardName == "" {
|
if shardName == "" {
|
||||||
return c.Random()
|
return c.Random()
|
||||||
}
|
}
|
||||||
@ -335,15 +349,15 @@ func (c *ringShards) GetByName(shardName string) (*ringShard, error) {
|
|||||||
c.mu.RLock()
|
c.mu.RLock()
|
||||||
defer c.mu.RUnlock()
|
defer c.mu.RUnlock()
|
||||||
|
|
||||||
return c.shards[shardName], nil
|
return c.shards.m[shardName], nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ringShards) Random() (*ringShard, error) {
|
func (c *ringSharding) Random() (*ringShard, error) {
|
||||||
return c.GetByKey(strconv.Itoa(rand.Int()))
|
return c.GetByKey(strconv.Itoa(rand.Int()))
|
||||||
}
|
}
|
||||||
|
|
||||||
// Heartbeat monitors state of each shard in the ring.
|
// Heartbeat monitors state of each shard in the ring.
|
||||||
func (c *ringShards) Heartbeat(ctx context.Context, frequency time.Duration) {
|
func (c *ringSharding) Heartbeat(ctx context.Context, frequency time.Duration) {
|
||||||
ticker := time.NewTicker(frequency)
|
ticker := time.NewTicker(frequency)
|
||||||
defer ticker.Stop()
|
defer ticker.Stop()
|
||||||
|
|
||||||
@ -371,14 +385,18 @@ func (c *ringShards) Heartbeat(ctx context.Context, frequency time.Duration) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// rebalance removes dead shards from the Ring.
|
// rebalance removes dead shards from the Ring.
|
||||||
func (c *ringShards) rebalance() {
|
func (c *ringSharding) rebalance() {
|
||||||
c.mu.RLock()
|
c.mu.RLock()
|
||||||
shards := c.shards
|
shards := c.shards
|
||||||
c.mu.RUnlock()
|
c.mu.RUnlock()
|
||||||
|
|
||||||
liveShards := make([]string, 0, len(shards))
|
if shards == nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
for name, shard := range shards {
|
liveShards := make([]string, 0, len(shards.m))
|
||||||
|
|
||||||
|
for name, shard := range shards.m {
|
||||||
if shard.IsUp() {
|
if shard.IsUp() {
|
||||||
liveShards = append(liveShards, name)
|
liveShards = append(liveShards, name)
|
||||||
}
|
}
|
||||||
@ -387,38 +405,21 @@ func (c *ringShards) rebalance() {
|
|||||||
hash := c.opt.NewConsistentHash(liveShards)
|
hash := c.opt.NewConsistentHash(liveShards)
|
||||||
|
|
||||||
c.mu.Lock()
|
c.mu.Lock()
|
||||||
c.hash = hash
|
if !c.closed {
|
||||||
c.numShard = len(liveShards)
|
c.hash = hash
|
||||||
|
c.numShard = len(liveShards)
|
||||||
|
}
|
||||||
c.mu.Unlock()
|
c.mu.Unlock()
|
||||||
}
|
}
|
||||||
|
|
||||||
// rebalanceLocked removes dead shards from the Ring and callers need to hold the locl
|
func (c *ringSharding) Len() int {
|
||||||
func (c *ringShards) rebalanceLocked() {
|
|
||||||
shards := c.shards
|
|
||||||
liveShards := make([]string, 0, len(shards))
|
|
||||||
|
|
||||||
for name, shard := range shards {
|
|
||||||
if shard.IsUp() {
|
|
||||||
liveShards = append(liveShards, name)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
hash := c.opt.NewConsistentHash(liveShards)
|
|
||||||
|
|
||||||
c.hash = hash
|
|
||||||
c.numShard = len(liveShards)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (c *ringShards) Len() int {
|
|
||||||
c.mu.RLock()
|
c.mu.RLock()
|
||||||
defer c.mu.RUnlock()
|
defer c.mu.RUnlock()
|
||||||
|
|
||||||
return c.numShard
|
return c.numShard
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ringShards) Close() error {
|
func (c *ringSharding) Close() error {
|
||||||
c.muClose.Lock()
|
|
||||||
defer c.muClose.Unlock()
|
|
||||||
c.mu.Lock()
|
c.mu.Lock()
|
||||||
defer c.mu.Unlock()
|
defer c.mu.Unlock()
|
||||||
|
|
||||||
@ -428,7 +429,8 @@ func (c *ringShards) Close() error {
|
|||||||
c.closed = true
|
c.closed = true
|
||||||
|
|
||||||
var firstErr error
|
var firstErr error
|
||||||
for _, shard := range c.shards {
|
|
||||||
|
for _, shard := range c.shards.list {
|
||||||
if err := shard.Client.Close(); err != nil && firstErr == nil {
|
if err := shard.Client.Close(); err != nil && firstErr == nil {
|
||||||
firstErr = err
|
firstErr = err
|
||||||
}
|
}
|
||||||
@ -437,20 +439,12 @@ func (c *ringShards) Close() error {
|
|||||||
c.hash = nil
|
c.hash = nil
|
||||||
c.shards = nil
|
c.shards = nil
|
||||||
c.numShard = 0
|
c.numShard = 0
|
||||||
c.list = nil
|
|
||||||
|
|
||||||
return firstErr
|
return firstErr
|
||||||
}
|
}
|
||||||
|
|
||||||
//------------------------------------------------------------------------------
|
//------------------------------------------------------------------------------
|
||||||
|
|
||||||
type ring struct {
|
|
||||||
opt *RingOptions
|
|
||||||
shards *ringShards
|
|
||||||
cmdsInfoCache *cmdsInfoCache //nolint:structcheck
|
|
||||||
heartbeatCancelFn context.CancelFunc
|
|
||||||
}
|
|
||||||
|
|
||||||
// Ring is a Redis client that uses consistent hashing to distribute
|
// Ring is a Redis client that uses consistent hashing to distribute
|
||||||
// keys across multiple Redis servers (shards). It's safe for
|
// keys across multiple Redis servers (shards). It's safe for
|
||||||
// concurrent use by multiple goroutines.
|
// concurrent use by multiple goroutines.
|
||||||
@ -466,7 +460,11 @@ type ring struct {
|
|||||||
// and can tolerate losing data when one of the servers dies.
|
// and can tolerate losing data when one of the servers dies.
|
||||||
// Otherwise you should use Redis Cluster.
|
// Otherwise you should use Redis Cluster.
|
||||||
type Ring struct {
|
type Ring struct {
|
||||||
*ring
|
opt *RingOptions
|
||||||
|
sharding *ringSharding
|
||||||
|
cmdsInfoCache *cmdsInfoCache
|
||||||
|
heartbeatCancelFn context.CancelFunc
|
||||||
|
|
||||||
cmdable
|
cmdable
|
||||||
hooks
|
hooks
|
||||||
}
|
}
|
||||||
@ -477,23 +475,21 @@ func NewRing(opt *RingOptions) *Ring {
|
|||||||
hbCtx, hbCancel := context.WithCancel(context.Background())
|
hbCtx, hbCancel := context.WithCancel(context.Background())
|
||||||
|
|
||||||
ring := Ring{
|
ring := Ring{
|
||||||
ring: &ring{
|
opt: opt,
|
||||||
opt: opt,
|
sharding: newRingSharding(opt),
|
||||||
shards: newRingShards(opt),
|
heartbeatCancelFn: hbCancel,
|
||||||
heartbeatCancelFn: hbCancel,
|
|
||||||
},
|
|
||||||
}
|
}
|
||||||
|
|
||||||
ring.cmdsInfoCache = newCmdsInfoCache(ring.cmdsInfo)
|
ring.cmdsInfoCache = newCmdsInfoCache(ring.cmdsInfo)
|
||||||
ring.cmdable = ring.Process
|
ring.cmdable = ring.Process
|
||||||
|
|
||||||
go ring.shards.Heartbeat(hbCtx, opt.HeartbeatFrequency)
|
go ring.sharding.Heartbeat(hbCtx, opt.HeartbeatFrequency)
|
||||||
|
|
||||||
return &ring
|
return &ring
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *Ring) SetAddrs(ctx context.Context, addrs map[string]string) {
|
func (c *Ring) SetAddrs(addrs map[string]string) {
|
||||||
c.shards.SetAddrs(addrs)
|
c.sharding.SetAddrs(addrs)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Do creates a Cmd from the args and processes the cmd.
|
// Do creates a Cmd from the args and processes the cmd.
|
||||||
@ -518,7 +514,7 @@ func (c *Ring) retryBackoff(attempt int) time.Duration {
|
|||||||
|
|
||||||
// PoolStats returns accumulated connection pool stats.
|
// PoolStats returns accumulated connection pool stats.
|
||||||
func (c *Ring) PoolStats() *PoolStats {
|
func (c *Ring) PoolStats() *PoolStats {
|
||||||
shards := c.shards.List()
|
shards := c.sharding.List()
|
||||||
var acc PoolStats
|
var acc PoolStats
|
||||||
for _, shard := range shards {
|
for _, shard := range shards {
|
||||||
s := shard.Client.connPool.Stats()
|
s := shard.Client.connPool.Stats()
|
||||||
@ -533,7 +529,7 @@ func (c *Ring) PoolStats() *PoolStats {
|
|||||||
|
|
||||||
// Len returns the current number of shards in the ring.
|
// Len returns the current number of shards in the ring.
|
||||||
func (c *Ring) Len() int {
|
func (c *Ring) Len() int {
|
||||||
return c.shards.Len()
|
return c.sharding.Len()
|
||||||
}
|
}
|
||||||
|
|
||||||
// Subscribe subscribes the client to the specified channels.
|
// Subscribe subscribes the client to the specified channels.
|
||||||
@ -542,7 +538,7 @@ func (c *Ring) Subscribe(ctx context.Context, channels ...string) *PubSub {
|
|||||||
panic("at least one channel is required")
|
panic("at least one channel is required")
|
||||||
}
|
}
|
||||||
|
|
||||||
shard, err := c.shards.GetByKey(channels[0])
|
shard, err := c.sharding.GetByKey(channels[0])
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// TODO: return PubSub with sticky error
|
// TODO: return PubSub with sticky error
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -556,7 +552,7 @@ func (c *Ring) PSubscribe(ctx context.Context, channels ...string) *PubSub {
|
|||||||
panic("at least one channel is required")
|
panic("at least one channel is required")
|
||||||
}
|
}
|
||||||
|
|
||||||
shard, err := c.shards.GetByKey(channels[0])
|
shard, err := c.sharding.GetByKey(channels[0])
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// TODO: return PubSub with sticky error
|
// TODO: return PubSub with sticky error
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -570,7 +566,7 @@ func (c *Ring) ForEachShard(
|
|||||||
ctx context.Context,
|
ctx context.Context,
|
||||||
fn func(ctx context.Context, client *Client) error,
|
fn func(ctx context.Context, client *Client) error,
|
||||||
) error {
|
) error {
|
||||||
shards := c.shards.List()
|
shards := c.sharding.List()
|
||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
errCh := make(chan error, 1)
|
errCh := make(chan error, 1)
|
||||||
for _, shard := range shards {
|
for _, shard := range shards {
|
||||||
@ -601,7 +597,7 @@ func (c *Ring) ForEachShard(
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (c *Ring) cmdsInfo(ctx context.Context) (map[string]*CommandInfo, error) {
|
func (c *Ring) cmdsInfo(ctx context.Context) (map[string]*CommandInfo, error) {
|
||||||
shards := c.shards.List()
|
shards := c.sharding.List()
|
||||||
var firstErr error
|
var firstErr error
|
||||||
for _, shard := range shards {
|
for _, shard := range shards {
|
||||||
cmdsInfo, err := shard.Client.Command(ctx).Result()
|
cmdsInfo, err := shard.Client.Command(ctx).Result()
|
||||||
@ -634,10 +630,10 @@ func (c *Ring) cmdShard(ctx context.Context, cmd Cmder) (*ringShard, error) {
|
|||||||
cmdInfo := c.cmdInfo(ctx, cmd.Name())
|
cmdInfo := c.cmdInfo(ctx, cmd.Name())
|
||||||
pos := cmdFirstKeyPos(cmd, cmdInfo)
|
pos := cmdFirstKeyPos(cmd, cmdInfo)
|
||||||
if pos == 0 {
|
if pos == 0 {
|
||||||
return c.shards.Random()
|
return c.sharding.Random()
|
||||||
}
|
}
|
||||||
firstKey := cmd.stringArg(pos)
|
firstKey := cmd.stringArg(pos)
|
||||||
return c.shards.GetByKey(firstKey)
|
return c.sharding.GetByKey(firstKey)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *Ring) process(ctx context.Context, cmd Cmder) error {
|
func (c *Ring) process(ctx context.Context, cmd Cmder) error {
|
||||||
@ -706,7 +702,7 @@ func (c *Ring) generalProcessPipeline(
|
|||||||
cmdInfo := c.cmdInfo(ctx, cmd.Name())
|
cmdInfo := c.cmdInfo(ctx, cmd.Name())
|
||||||
hash := cmd.stringArg(cmdFirstKeyPos(cmd, cmdInfo))
|
hash := cmd.stringArg(cmdFirstKeyPos(cmd, cmdInfo))
|
||||||
if hash != "" {
|
if hash != "" {
|
||||||
hash = c.shards.Hash(hash)
|
hash = c.sharding.Hash(hash)
|
||||||
}
|
}
|
||||||
cmdsMap[hash] = append(cmdsMap[hash], cmd)
|
cmdsMap[hash] = append(cmdsMap[hash], cmd)
|
||||||
}
|
}
|
||||||
@ -729,7 +725,7 @@ func (c *Ring) processShardPipeline(
|
|||||||
ctx context.Context, hash string, cmds []Cmder, tx bool,
|
ctx context.Context, hash string, cmds []Cmder, tx bool,
|
||||||
) error {
|
) error {
|
||||||
// TODO: retry?
|
// TODO: retry?
|
||||||
shard, err := c.shards.GetByName(hash)
|
shard, err := c.sharding.GetByName(hash)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
setCmdsErr(cmds, err)
|
setCmdsErr(cmds, err)
|
||||||
return err
|
return err
|
||||||
@ -749,7 +745,7 @@ func (c *Ring) Watch(ctx context.Context, fn func(*Tx) error, keys ...string) er
|
|||||||
var shards []*ringShard
|
var shards []*ringShard
|
||||||
for _, key := range keys {
|
for _, key := range keys {
|
||||||
if key != "" {
|
if key != "" {
|
||||||
shard, err := c.shards.GetByKey(hashtag.Key(key))
|
shard, err := c.sharding.GetByKey(hashtag.Key(key))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -781,5 +777,5 @@ func (c *Ring) Watch(ctx context.Context, fn func(*Tx) error, keys ...string) er
|
|||||||
func (c *Ring) Close() error {
|
func (c *Ring) Close() error {
|
||||||
c.heartbeatCancelFn()
|
c.heartbeatCancelFn()
|
||||||
|
|
||||||
return c.shards.Close()
|
return c.sharding.Close()
|
||||||
}
|
}
|
||||||
|
34
ring_test.go
34
ring_test.go
@ -117,26 +117,25 @@ var _ = Describe("Redis Ring", func() {
|
|||||||
It("downscale shard and check reuse shard, upscale shard and check reuse", func() {
|
It("downscale shard and check reuse shard, upscale shard and check reuse", func() {
|
||||||
Expect(ring.Len(), 2)
|
Expect(ring.Len(), 2)
|
||||||
|
|
||||||
wantShard := ring.GetAddr("ringShardOne")
|
wantShard := ring.ShardByName("ringShardOne")
|
||||||
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
ring.SetAddrs(ctx, map[string]string{
|
ring.SetAddrs(map[string]string{
|
||||||
"ringShardOne": ":" + ringShard1Port,
|
"ringShardOne": ":" + ringShard1Port,
|
||||||
})
|
})
|
||||||
Expect(ring.Len(), 1)
|
Expect(ring.Len(), 1)
|
||||||
gotShard := ring.GetAddr("ringShardOne")
|
gotShard := ring.ShardByName("ringShardOne")
|
||||||
Expect(gotShard).To(Equal(wantShard))
|
Expect(gotShard).To(Equal(wantShard))
|
||||||
|
|
||||||
ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second)
|
ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
ring.SetAddrs(ctx, map[string]string{
|
ring.SetAddrs(map[string]string{
|
||||||
"ringShardOne": ":" + ringShard1Port,
|
"ringShardOne": ":" + ringShard1Port,
|
||||||
"ringShardTwo": ":" + ringShard2Port,
|
"ringShardTwo": ":" + ringShard2Port,
|
||||||
})
|
})
|
||||||
Expect(ring.Len(), 2)
|
Expect(ring.Len(), 2)
|
||||||
gotShard = ring.GetAddr("ringShardOne")
|
gotShard = ring.ShardByName("ringShardOne")
|
||||||
Expect(gotShard).To(Equal(wantShard))
|
Expect(gotShard).To(Equal(wantShard))
|
||||||
|
|
||||||
})
|
})
|
||||||
|
|
||||||
It("uses 3 shards after setting it to 3 shards", func() {
|
It("uses 3 shards after setting it to 3 shards", func() {
|
||||||
@ -149,42 +148,41 @@ var _ = Describe("Redis Ring", func() {
|
|||||||
|
|
||||||
shardName1 := "ringShardOne"
|
shardName1 := "ringShardOne"
|
||||||
shardAddr1 := ":" + ringShard1Port
|
shardAddr1 := ":" + ringShard1Port
|
||||||
wantShard1 := ring.GetAddr(shardName1)
|
wantShard1 := ring.ShardByName(shardName1)
|
||||||
shardName2 := "ringShardTwo"
|
shardName2 := "ringShardTwo"
|
||||||
shardAddr2 := ":" + ringShard2Port
|
shardAddr2 := ":" + ringShard2Port
|
||||||
wantShard2 := ring.GetAddr(shardName2)
|
wantShard2 := ring.ShardByName(shardName2)
|
||||||
shardName3 := "ringShardThree"
|
shardName3 := "ringShardThree"
|
||||||
shardAddr3 := ":" + ringShard3Port
|
shardAddr3 := ":" + ringShard3Port
|
||||||
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
ring.SetAddrs(ctx, map[string]string{
|
ring.SetAddrs(map[string]string{
|
||||||
shardName1: shardAddr1,
|
shardName1: shardAddr1,
|
||||||
shardName2: shardAddr2,
|
shardName2: shardAddr2,
|
||||||
shardName3: shardAddr3,
|
shardName3: shardAddr3,
|
||||||
})
|
})
|
||||||
Expect(ring.Len(), 3)
|
Expect(ring.Len(), 3)
|
||||||
gotShard1 := ring.GetAddr(shardName1)
|
gotShard1 := ring.ShardByName(shardName1)
|
||||||
gotShard2 := ring.GetAddr(shardName2)
|
gotShard2 := ring.ShardByName(shardName2)
|
||||||
gotShard3 := ring.GetAddr(shardName3)
|
gotShard3 := ring.ShardByName(shardName3)
|
||||||
Expect(gotShard1).To(Equal(wantShard1))
|
Expect(gotShard1).To(Equal(wantShard1))
|
||||||
Expect(gotShard2).To(Equal(wantShard2))
|
Expect(gotShard2).To(Equal(wantShard2))
|
||||||
Expect(gotShard3).ToNot(BeNil())
|
Expect(gotShard3).ToNot(BeNil())
|
||||||
|
|
||||||
ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second)
|
ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
ring.SetAddrs(ctx, map[string]string{
|
ring.SetAddrs(map[string]string{
|
||||||
shardName1: shardAddr1,
|
shardName1: shardAddr1,
|
||||||
shardName2: shardAddr2,
|
shardName2: shardAddr2,
|
||||||
})
|
})
|
||||||
Expect(ring.Len(), 2)
|
Expect(ring.Len(), 2)
|
||||||
gotShard1 = ring.GetAddr(shardName1)
|
gotShard1 = ring.ShardByName(shardName1)
|
||||||
gotShard2 = ring.GetAddr(shardName2)
|
gotShard2 = ring.ShardByName(shardName2)
|
||||||
gotShard3 = ring.GetAddr(shardName3)
|
gotShard3 = ring.ShardByName(shardName3)
|
||||||
Expect(gotShard1).To(Equal(wantShard1))
|
Expect(gotShard1).To(Equal(wantShard1))
|
||||||
Expect(gotShard2).To(Equal(wantShard2))
|
Expect(gotShard2).To(Equal(wantShard2))
|
||||||
Expect(gotShard3).To(BeNil())
|
Expect(gotShard3).To(BeNil())
|
||||||
})
|
})
|
||||||
|
|
||||||
})
|
})
|
||||||
Describe("pipeline", func() {
|
Describe("pipeline", func() {
|
||||||
It("doesn't panic closed ring, returns error", func() {
|
It("doesn't panic closed ring, returns error", func() {
|
||||||
@ -263,7 +261,7 @@ var _ = Describe("Redis Ring", func() {
|
|||||||
Describe("new client callback", func() {
|
Describe("new client callback", func() {
|
||||||
It("can be initialized with a new client callback", func() {
|
It("can be initialized with a new client callback", func() {
|
||||||
opts := redisRingOptions()
|
opts := redisRingOptions()
|
||||||
opts.NewClient = func(name string, opt *redis.Options) *redis.Client {
|
opts.NewClient = func(opt *redis.Options) *redis.Client {
|
||||||
opt.Username = "username1"
|
opt.Username = "username1"
|
||||||
opt.Password = "password1"
|
opt.Password = "password1"
|
||||||
return redis.NewClient(opt)
|
return redis.NewClient(opt)
|
||||||
|
Loading…
x
Reference in New Issue
Block a user