mirror of
https://github.com/HDT3213/godis.git
synced 2025-10-06 09:17:10 +08:00
raft cluster core
This commit is contained in:
73
cluster/core/command.go
Normal file
73
cluster/core/command.go
Normal file
@@ -0,0 +1,73 @@
|
|||||||
|
package core
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"runtime/debug"
|
||||||
|
"strings"
|
||||||
|
|
||||||
|
"github.com/hdt3213/godis/config"
|
||||||
|
"github.com/hdt3213/godis/database"
|
||||||
|
"github.com/hdt3213/godis/interface/redis"
|
||||||
|
"github.com/hdt3213/godis/lib/logger"
|
||||||
|
"github.com/hdt3213/godis/redis/protocol"
|
||||||
|
)
|
||||||
|
|
||||||
|
// CmdLine is alias for [][]byte, represents a command line
|
||||||
|
type CmdLine = [][]byte
|
||||||
|
|
||||||
|
// CmdFunc represents the handler of a redis command
|
||||||
|
type CmdFunc func(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply
|
||||||
|
|
||||||
|
var commands = make(map[string]CmdFunc)
|
||||||
|
|
||||||
|
// RegisterCmd add command handler into cluster
|
||||||
|
func RegisterCmd(name string, cmd CmdFunc) {
|
||||||
|
name = strings.ToLower(name)
|
||||||
|
commands[name] = cmd
|
||||||
|
}
|
||||||
|
|
||||||
|
// Exec executes command on cluster
|
||||||
|
func (cluster *Cluster) Exec(c redis.Connection, cmdLine [][]byte) (result redis.Reply) {
|
||||||
|
defer func() {
|
||||||
|
if err := recover(); err != nil {
|
||||||
|
logger.Warn(fmt.Sprintf("error occurs: %v\n%s", err, string(debug.Stack())))
|
||||||
|
result = &protocol.UnknownErrReply{}
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
cmdName := strings.ToLower(string(cmdLine[0]))
|
||||||
|
if cmdName == "auth" {
|
||||||
|
return database.Auth(c, cmdLine[1:])
|
||||||
|
}
|
||||||
|
if !isAuthenticated(c) {
|
||||||
|
return protocol.MakeErrReply("NOAUTH Authentication required")
|
||||||
|
}
|
||||||
|
cmdFunc, ok := commands[cmdName]
|
||||||
|
if !ok {
|
||||||
|
return protocol.MakeErrReply("ERR unknown command '" + cmdName + "', or not supported in cluster mode")
|
||||||
|
}
|
||||||
|
return cmdFunc(cluster, c, cmdLine)
|
||||||
|
}
|
||||||
|
|
||||||
|
func isAuthenticated(c redis.Connection) bool {
|
||||||
|
if config.Properties.RequirePass == "" {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
return c.GetPassword() == config.Properties.RequirePass
|
||||||
|
}
|
||||||
|
|
||||||
|
func RegisterDefaultCmd(name string) {
|
||||||
|
RegisterCmd(name, DefaultFunc)
|
||||||
|
}
|
||||||
|
|
||||||
|
// relay command to responsible peer, and return its protocol to client
|
||||||
|
func DefaultFunc(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||||
|
key := string(args[1])
|
||||||
|
slotId := GetSlot(key)
|
||||||
|
peer := cluster.PickNode(slotId)
|
||||||
|
if peer == cluster.SelfID() {
|
||||||
|
// to self db
|
||||||
|
//return cluster.db.Exec(c, cmdLine)
|
||||||
|
return cluster.db.Exec(c, args)
|
||||||
|
}
|
||||||
|
return cluster.Relay(peer, c, args)
|
||||||
|
}
|
179
cluster/core/connection.go
Normal file
179
cluster/core/connection.go
Normal file
@@ -0,0 +1,179 @@
|
|||||||
|
package core
|
||||||
|
|
||||||
|
import (
|
||||||
|
"errors"
|
||||||
|
"fmt"
|
||||||
|
"net"
|
||||||
|
|
||||||
|
"github.com/hdt3213/godis/config"
|
||||||
|
"github.com/hdt3213/godis/datastruct/dict"
|
||||||
|
"github.com/hdt3213/godis/interface/redis"
|
||||||
|
"github.com/hdt3213/godis/lib/logger"
|
||||||
|
"github.com/hdt3213/godis/lib/pool"
|
||||||
|
"github.com/hdt3213/godis/lib/utils"
|
||||||
|
"github.com/hdt3213/godis/redis/client"
|
||||||
|
"github.com/hdt3213/godis/redis/parser"
|
||||||
|
"github.com/hdt3213/godis/redis/protocol"
|
||||||
|
)
|
||||||
|
|
||||||
|
// ConnectionFactory manages connection with peer nodes in cluster
|
||||||
|
type ConnectionFactory interface {
|
||||||
|
BorrowPeerClient(peerAddr string) (peerClient, error)
|
||||||
|
ReturnPeerClient(peerClient peerClient) error
|
||||||
|
NewStream(peerAddr string, cmdLine CmdLine) (peerStream, error)
|
||||||
|
Close() error
|
||||||
|
}
|
||||||
|
|
||||||
|
// peerClient represents a
|
||||||
|
type peerClient interface {
|
||||||
|
RemoteAddress() string
|
||||||
|
Send(args [][]byte) redis.Reply
|
||||||
|
}
|
||||||
|
|
||||||
|
type peerStream interface {
|
||||||
|
Stream() <-chan *parser.Payload
|
||||||
|
Close() error
|
||||||
|
}
|
||||||
|
|
||||||
|
type defaultClientFactory struct {
|
||||||
|
nodeConnections dict.Dict // map[string]*pool.Pool
|
||||||
|
}
|
||||||
|
|
||||||
|
var connectionPoolConfig = pool.Config{
|
||||||
|
MaxIdle: 1,
|
||||||
|
MaxActive: 16,
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewFactory() ConnectionFactory {
|
||||||
|
return &defaultClientFactory{
|
||||||
|
nodeConnections: dict.MakeSimple(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewPeerClient creats a new client, no need to return this client
|
||||||
|
func (factory *defaultClientFactory) NewPeerClient(peerAddr string) (peerClient, error) {
|
||||||
|
c, err := client.MakeClient(peerAddr)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
c.Start()
|
||||||
|
// all peers of cluster should use the same password
|
||||||
|
if config.Properties.RequirePass != "" {
|
||||||
|
authResp := c.Send(utils.ToCmdLine("AUTH", config.Properties.RequirePass))
|
||||||
|
if !protocol.IsOKReply(authResp) {
|
||||||
|
return nil, fmt.Errorf("auth failed, resp: %s", string(authResp.ToBytes()))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return c, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetPeerClient gets a client with peer form pool
|
||||||
|
func (factory *defaultClientFactory) BorrowPeerClient(peerAddr string) (peerClient, error) {
|
||||||
|
var connectionPool *pool.Pool
|
||||||
|
raw, ok := factory.nodeConnections.Get(peerAddr)
|
||||||
|
if !ok {
|
||||||
|
creator := func() (interface{}, error) {
|
||||||
|
return factory.NewPeerClient(peerAddr)
|
||||||
|
}
|
||||||
|
finalizer := func(x interface{}) {
|
||||||
|
logger.Debug("destroy client")
|
||||||
|
cli, ok := x.(client.Client)
|
||||||
|
if !ok {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
cli.Close()
|
||||||
|
}
|
||||||
|
connectionPool = pool.New(creator, finalizer, connectionPoolConfig)
|
||||||
|
factory.nodeConnections.Put(peerAddr, connectionPool)
|
||||||
|
} else {
|
||||||
|
connectionPool = raw.(*pool.Pool)
|
||||||
|
}
|
||||||
|
raw, err := connectionPool.Get()
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
conn, ok := raw.(*client.Client)
|
||||||
|
if !ok {
|
||||||
|
return nil, errors.New("connection pool make wrong type")
|
||||||
|
}
|
||||||
|
return conn, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cluster *Cluster) BorrowLeaderClient() (peerClient, error) {
|
||||||
|
leaderAddr := cluster.raftNode.GetLeaderRedisAddress()
|
||||||
|
return cluster.connections.BorrowPeerClient(leaderAddr)
|
||||||
|
}
|
||||||
|
|
||||||
|
// ReturnPeerClient returns client to pool
|
||||||
|
func (factory *defaultClientFactory) ReturnPeerClient(peerClient peerClient) error {
|
||||||
|
raw, ok := factory.nodeConnections.Get(peerClient.RemoteAddress())
|
||||||
|
if !ok {
|
||||||
|
return errors.New("connection pool not found")
|
||||||
|
}
|
||||||
|
raw.(*pool.Pool).Put(peerClient)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
type tcpStream struct {
|
||||||
|
conn net.Conn
|
||||||
|
ch <-chan *parser.Payload
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *tcpStream) Stream() <-chan *parser.Payload {
|
||||||
|
return s.ch
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *tcpStream) Close() error {
|
||||||
|
return s.conn.Close()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (factory *defaultClientFactory) NewStream(peerAddr string, cmdLine CmdLine) (peerStream, error) {
|
||||||
|
// todo: reuse connection
|
||||||
|
conn, err := net.Dial("tcp", peerAddr)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("connect with %s failed: %v", peerAddr, err)
|
||||||
|
}
|
||||||
|
ch := parser.ParseStream(conn)
|
||||||
|
send2node := func(cmdLine CmdLine) redis.Reply {
|
||||||
|
req := protocol.MakeMultiBulkReply(cmdLine)
|
||||||
|
_, err := conn.Write(req.ToBytes())
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply(err.Error())
|
||||||
|
}
|
||||||
|
resp := <-ch
|
||||||
|
if resp.Err != nil {
|
||||||
|
return protocol.MakeErrReply(resp.Err.Error())
|
||||||
|
}
|
||||||
|
return resp.Data
|
||||||
|
}
|
||||||
|
if config.Properties.RequirePass != "" {
|
||||||
|
authResp := send2node(utils.ToCmdLine("AUTH", config.Properties.RequirePass))
|
||||||
|
if !protocol.IsOKReply(authResp) {
|
||||||
|
return nil, fmt.Errorf("auth failed, resp: %s", string(authResp.ToBytes()))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
req := protocol.MakeMultiBulkReply(cmdLine)
|
||||||
|
_, err = conn.Write(req.ToBytes())
|
||||||
|
if err != nil {
|
||||||
|
return nil, protocol.MakeErrReply("send cmdLine failed: " + err.Error())
|
||||||
|
}
|
||||||
|
return &tcpStream{
|
||||||
|
conn: conn,
|
||||||
|
ch: ch,
|
||||||
|
}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func newDefaultClientFactory() *defaultClientFactory {
|
||||||
|
return &defaultClientFactory{
|
||||||
|
nodeConnections: dict.MakeConcurrent(1),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (factory *defaultClientFactory) Close() error {
|
||||||
|
factory.nodeConnections.ForEach(func(key string, val interface{}) bool {
|
||||||
|
val.(*pool.Pool).Close()
|
||||||
|
return true
|
||||||
|
})
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
84
cluster/core/connection_inmem.go
Normal file
84
cluster/core/connection_inmem.go
Normal file
@@ -0,0 +1,84 @@
|
|||||||
|
package core
|
||||||
|
|
||||||
|
import (
|
||||||
|
"sync"
|
||||||
|
|
||||||
|
"github.com/hdt3213/godis/interface/redis"
|
||||||
|
"github.com/hdt3213/godis/redis/connection"
|
||||||
|
"github.com/hdt3213/godis/redis/parser"
|
||||||
|
)
|
||||||
|
|
||||||
|
type InMemConnectionFactory struct {
|
||||||
|
nodes map[string]*Cluster
|
||||||
|
mu sync.Mutex
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewInMemConnectionFactory() *InMemConnectionFactory {
|
||||||
|
return &InMemConnectionFactory{
|
||||||
|
nodes: make(map[string]*Cluster),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type InMemClient struct {
|
||||||
|
addr string
|
||||||
|
cluster *Cluster
|
||||||
|
}
|
||||||
|
|
||||||
|
// RemoteAddress implements peerClient.
|
||||||
|
func (c *InMemClient) RemoteAddress() string {
|
||||||
|
return c.addr
|
||||||
|
}
|
||||||
|
|
||||||
|
// Send implements peerClient.
|
||||||
|
func (c *InMemClient) Send(args [][]byte) redis.Reply {
|
||||||
|
fakeConn := connection.NewFakeConn()
|
||||||
|
return c.cluster.Exec(fakeConn, args)
|
||||||
|
}
|
||||||
|
|
||||||
|
type InMemStream struct {
|
||||||
|
conn *connection.FakeConn
|
||||||
|
cluster *Cluster
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *InMemStream) Stream() <-chan *parser.Payload {
|
||||||
|
return parser.ParseStream(c.conn)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *InMemStream) Close() error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (factory *InMemConnectionFactory) NewPeerClient(peerAddr string) (peerClient, error) {
|
||||||
|
factory.mu.Lock()
|
||||||
|
cluster := factory.nodes[peerAddr]
|
||||||
|
factory.mu.Unlock()
|
||||||
|
return &InMemClient{
|
||||||
|
addr: peerAddr,
|
||||||
|
cluster: cluster,
|
||||||
|
}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (factory *InMemConnectionFactory) BorrowPeerClient(peerAddr string) (peerClient, error) {
|
||||||
|
return factory.NewPeerClient(peerAddr)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (factory *InMemConnectionFactory) ReturnPeerClient(peerClient peerClient) error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (factory *InMemConnectionFactory) NewStream(peerAddr string, cmdLine CmdLine) (peerStream, error) {
|
||||||
|
factory.mu.Lock()
|
||||||
|
cluster := factory.nodes[peerAddr]
|
||||||
|
factory.mu.Unlock()
|
||||||
|
conn := connection.NewFakeConn()
|
||||||
|
reply := cluster.Exec(conn, cmdLine)
|
||||||
|
conn.Write(reply.ToBytes()) // append response at the end
|
||||||
|
return &InMemStream{
|
||||||
|
conn: conn,
|
||||||
|
cluster: cluster,
|
||||||
|
}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (factory *InMemConnectionFactory) Close() error {
|
||||||
|
return nil
|
||||||
|
}
|
132
cluster/core/core.go
Normal file
132
cluster/core/core.go
Normal file
@@ -0,0 +1,132 @@
|
|||||||
|
package core
|
||||||
|
|
||||||
|
import (
|
||||||
|
"sync"
|
||||||
|
|
||||||
|
"github.com/hdt3213/godis/cluster/raft"
|
||||||
|
dbimpl "github.com/hdt3213/godis/database"
|
||||||
|
"github.com/hdt3213/godis/datastruct/set"
|
||||||
|
"github.com/hdt3213/godis/interface/database"
|
||||||
|
"github.com/hdt3213/godis/lib/utils"
|
||||||
|
"github.com/hdt3213/godis/redis/protocol"
|
||||||
|
)
|
||||||
|
|
||||||
|
type Cluster struct {
|
||||||
|
raftNode *raft.Node
|
||||||
|
db database.DBEngine
|
||||||
|
connections ConnectionFactory
|
||||||
|
config *Config
|
||||||
|
|
||||||
|
slotsManager *slotsManager
|
||||||
|
rebalanceManger *rebalanceManager
|
||||||
|
}
|
||||||
|
|
||||||
|
type Config struct {
|
||||||
|
raft.RaftConfig
|
||||||
|
StartAsSeed bool
|
||||||
|
JoinAddress string
|
||||||
|
connectionStub ConnectionFactory // for test
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *Cluster) SelfID() string {
|
||||||
|
return c.raftNode.Cfg.ID()
|
||||||
|
}
|
||||||
|
|
||||||
|
// slotsManager 负责管理当前 node 上的 slot
|
||||||
|
type slotsManager struct {
|
||||||
|
mu *sync.RWMutex
|
||||||
|
slots map[uint32]*slotStatus // 记录当前node上的 slot
|
||||||
|
importingTask *raft.MigratingTask
|
||||||
|
}
|
||||||
|
|
||||||
|
const (
|
||||||
|
slotStateHosting = iota
|
||||||
|
slotStateImporting
|
||||||
|
slotStateExporting
|
||||||
|
)
|
||||||
|
|
||||||
|
type slotStatus struct {
|
||||||
|
mu *sync.RWMutex
|
||||||
|
state int
|
||||||
|
keys *set.Set // 记录当前 slot 上的 key
|
||||||
|
|
||||||
|
exportSnapshot *set.Set // 开始传输时拷贝 slot 中的 key, 避免并发并发
|
||||||
|
dirtyKeys *set.Set // 传输开始后被修改的key, 在传输结束阶段需要重传一遍
|
||||||
|
}
|
||||||
|
|
||||||
|
func newSlotsManager() *slotsManager {
|
||||||
|
return &slotsManager{
|
||||||
|
mu: &sync.RWMutex{},
|
||||||
|
slots: map[uint32]*slotStatus{},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ssm *slotsManager) getSlot(index uint32) *slotStatus {
|
||||||
|
ssm.mu.RLock()
|
||||||
|
slot := ssm.slots[index]
|
||||||
|
ssm.mu.RUnlock()
|
||||||
|
if slot != nil {
|
||||||
|
return slot
|
||||||
|
}
|
||||||
|
ssm.mu.Lock()
|
||||||
|
defer ssm.mu.Unlock()
|
||||||
|
// check-lock-check
|
||||||
|
slot = ssm.slots[index]
|
||||||
|
if slot != nil {
|
||||||
|
return slot
|
||||||
|
}
|
||||||
|
slot = &slotStatus{
|
||||||
|
state: slotStateHosting,
|
||||||
|
keys: set.Make(),
|
||||||
|
mu: &sync.RWMutex{},
|
||||||
|
}
|
||||||
|
ssm.slots[index] = slot
|
||||||
|
return slot
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewCluster(cfg *Config) (*Cluster, error) {
|
||||||
|
var connections ConnectionFactory
|
||||||
|
if cfg.connectionStub != nil {
|
||||||
|
connections = cfg.connectionStub
|
||||||
|
} else {
|
||||||
|
connections = newDefaultClientFactory()
|
||||||
|
}
|
||||||
|
db := dbimpl.NewStandaloneServer()
|
||||||
|
raftNode, err := raft.StartNode(&cfg.RaftConfig)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
hasState, err := raftNode.HasExistingState()
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if !hasState {
|
||||||
|
if cfg.StartAsSeed {
|
||||||
|
err = raftNode.BootstrapCluster(SlotCount)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// join cluster
|
||||||
|
conn, err := connections.BorrowPeerClient(cfg.JoinAddress)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
result := conn.Send(utils.ToCmdLine(joinClusterCommand, cfg.RedisAdvertiseAddr, cfg.RaftAdvertiseAddr))
|
||||||
|
if err := protocol.Try2ErrorReply(result); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
cluster := &Cluster{
|
||||||
|
raftNode: raftNode,
|
||||||
|
db: db,
|
||||||
|
connections: connections,
|
||||||
|
config: cfg,
|
||||||
|
rebalanceManger: newRebalanceManager(),
|
||||||
|
slotsManager: newSlotsManager(),
|
||||||
|
}
|
||||||
|
cluster.injectInsertCallback()
|
||||||
|
cluster.injectDeleteCallback()
|
||||||
|
return cluster, nil
|
||||||
|
}
|
103
cluster/core/core_test.go
Normal file
103
cluster/core/core_test.go
Normal file
@@ -0,0 +1,103 @@
|
|||||||
|
package core
|
||||||
|
|
||||||
|
import (
|
||||||
|
"os"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/hdt3213/godis/cluster/raft"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestClusterBootstrap(t *testing.T) {
|
||||||
|
// start leader
|
||||||
|
leaderDir := "test/0"
|
||||||
|
os.RemoveAll(leaderDir)
|
||||||
|
os.MkdirAll(leaderDir, 0777)
|
||||||
|
defer func() {
|
||||||
|
os.RemoveAll(leaderDir)
|
||||||
|
}()
|
||||||
|
// connection stub
|
||||||
|
connections := NewInMemConnectionFactory()
|
||||||
|
leaderCfg := &Config{
|
||||||
|
RaftConfig: raft.RaftConfig{
|
||||||
|
RedisAdvertiseAddr: "127.0.0.1:6399",
|
||||||
|
RaftListenAddr: "127.0.0.1:16666",
|
||||||
|
RaftAdvertiseAddr: "127.0.0.1:16666",
|
||||||
|
Dir: leaderDir,
|
||||||
|
},
|
||||||
|
StartAsSeed: true,
|
||||||
|
connectionStub: connections,
|
||||||
|
}
|
||||||
|
leader, err := NewCluster(leaderCfg)
|
||||||
|
if err != nil {
|
||||||
|
t.Error(err)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
connections.nodes[leaderCfg.RedisAdvertiseAddr] = leader
|
||||||
|
|
||||||
|
// start follower
|
||||||
|
followerDir := "test/1"
|
||||||
|
os.RemoveAll(followerDir)
|
||||||
|
os.MkdirAll(followerDir, 0777)
|
||||||
|
defer func() {
|
||||||
|
os.RemoveAll(followerDir)
|
||||||
|
}()
|
||||||
|
followerCfg := &Config{
|
||||||
|
RaftConfig: raft.RaftConfig{
|
||||||
|
RedisAdvertiseAddr: "127.0.0.1:6499",
|
||||||
|
RaftListenAddr: "127.0.0.1:16667",
|
||||||
|
RaftAdvertiseAddr: "127.0.0.1:16667",
|
||||||
|
Dir: followerDir,
|
||||||
|
},
|
||||||
|
StartAsSeed: false,
|
||||||
|
JoinAddress: leaderCfg.RedisAdvertiseAddr,
|
||||||
|
connectionStub: connections,
|
||||||
|
}
|
||||||
|
follower, err := NewCluster(followerCfg)
|
||||||
|
if err != nil {
|
||||||
|
t.Error(err)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
connections.nodes[followerCfg.RedisAdvertiseAddr] = follower
|
||||||
|
|
||||||
|
_ = follower.SelfID()
|
||||||
|
// check nodes
|
||||||
|
joined := false
|
||||||
|
for i := 0; i < 10; i++ {
|
||||||
|
nodes, err := leader.raftNode.GetNodes()
|
||||||
|
if err != nil {
|
||||||
|
t.Log(err)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
if len(nodes) == 2 {
|
||||||
|
t.Log("join success")
|
||||||
|
joined = true
|
||||||
|
break
|
||||||
|
}
|
||||||
|
time.Sleep(time.Second)
|
||||||
|
}
|
||||||
|
if !joined {
|
||||||
|
t.Error("join failed")
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
// rebalance
|
||||||
|
leader.doRebalance()
|
||||||
|
time.Sleep(2 * time.Second)
|
||||||
|
for i := 0; i < 1000; i++ {
|
||||||
|
leaderSuccess := false
|
||||||
|
leader.raftNode.FSM.WithReadLock(func(fsm *raft.FSM) {
|
||||||
|
leaderSlots := len(fsm.Node2Slot[leaderCfg.RedisAdvertiseAddr])
|
||||||
|
followerSlots := len(fsm.Node2Slot[followerCfg.RedisAdvertiseAddr])
|
||||||
|
if len(fsm.Migratings) == 0 && leaderSlots > 0 && followerSlots > 0 {
|
||||||
|
leaderSuccess = true
|
||||||
|
}
|
||||||
|
})
|
||||||
|
if leaderSuccess {
|
||||||
|
t.Log("rebalance success")
|
||||||
|
break
|
||||||
|
} else {
|
||||||
|
time.Sleep(time.Second)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
246
cluster/core/leader.go
Normal file
246
cluster/core/leader.go
Normal file
@@ -0,0 +1,246 @@
|
|||||||
|
package core
|
||||||
|
|
||||||
|
import (
|
||||||
|
"errors"
|
||||||
|
"fmt"
|
||||||
|
"math"
|
||||||
|
"strconv"
|
||||||
|
"sync"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/hdt3213/godis/cluster/raft"
|
||||||
|
"github.com/hdt3213/godis/interface/redis"
|
||||||
|
"github.com/hdt3213/godis/lib/logger"
|
||||||
|
"github.com/hdt3213/godis/lib/utils"
|
||||||
|
"github.com/hdt3213/godis/redis/protocol"
|
||||||
|
)
|
||||||
|
|
||||||
|
const joinClusterCommand = "cluster.join"
|
||||||
|
const migrationChangeRouteCommand = "cluster.migration.changeroute"
|
||||||
|
|
||||||
|
func init() {
|
||||||
|
RegisterCmd(joinClusterCommand, execJoin)
|
||||||
|
RegisterCmd(migrationChangeRouteCommand, execMigrationChangeRoute)
|
||||||
|
}
|
||||||
|
|
||||||
|
// execJoin handles cluster-join command
|
||||||
|
// format: cluster-join redisAddress (advertised)raftAddress
|
||||||
|
func execJoin(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||||
|
if len(cmdLine) != 3 {
|
||||||
|
return protocol.MakeArgNumErrReply(joinClusterCommand)
|
||||||
|
}
|
||||||
|
state := cluster.raftNode.State()
|
||||||
|
if state != raft.Leader {
|
||||||
|
// I am not leader, forward request to leader
|
||||||
|
leaderConn, err := cluster.BorrowLeaderClient()
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply(err.Error())
|
||||||
|
}
|
||||||
|
defer cluster.connections.ReturnPeerClient(leaderConn)
|
||||||
|
return leaderConn.Send(cmdLine)
|
||||||
|
}
|
||||||
|
// self node is leader
|
||||||
|
redisAddr := string(cmdLine[1])
|
||||||
|
raftAddr := string(cmdLine[2])
|
||||||
|
err := cluster.raftNode.HandleJoin(redisAddr, raftAddr)
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply(err.Error())
|
||||||
|
}
|
||||||
|
// join sucees, rebalance node
|
||||||
|
return protocol.MakeOkReply()
|
||||||
|
}
|
||||||
|
|
||||||
|
type rebalanceManager struct {
|
||||||
|
mu *sync.Mutex
|
||||||
|
}
|
||||||
|
|
||||||
|
func newRebalanceManager() *rebalanceManager {
|
||||||
|
return &rebalanceManager{
|
||||||
|
mu: &sync.Mutex{},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cluster *Cluster) doRebalance() {
|
||||||
|
cluster.rebalanceManger.mu.Lock()
|
||||||
|
defer cluster.rebalanceManger.mu.Unlock()
|
||||||
|
pendingTasks, err := cluster.makeRebalancePlan()
|
||||||
|
if err != nil {
|
||||||
|
logger.Errorf("makeRebalancePlan err: %v", err)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
logger.Infof("rebalance plan generated, contains %d tasks", len(pendingTasks))
|
||||||
|
if len(pendingTasks) == 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
for _, task := range pendingTasks {
|
||||||
|
err := cluster.triggerMigrationTask(task)
|
||||||
|
if err != nil {
|
||||||
|
logger.Errorf("triggerMigrationTask err: %v", err)
|
||||||
|
} else {
|
||||||
|
logger.Infof("triggerMigrationTask %s success", task.ID)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
// triggerRebalanceTask start a rebalance task
|
||||||
|
// only leader can do
|
||||||
|
func (cluster *Cluster) triggerMigrationTask(task *raft.MigratingTask) error {
|
||||||
|
// propose migration
|
||||||
|
_, err := cluster.raftNode.Propose(&raft.LogEntry{
|
||||||
|
Event: raft.EventStartMigrate,
|
||||||
|
MigratingTask: task,
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
return fmt.Errorf("propose EventStartMigrate %s failed: %v", task.ID, err)
|
||||||
|
}
|
||||||
|
logger.Infof("propose EventStartMigrate %s success", task.ID)
|
||||||
|
|
||||||
|
cmdLine := utils.ToCmdLine(startMigrationCommand, task.ID, task.SrcNode)
|
||||||
|
for _, slotId := range task.Slots {
|
||||||
|
slotIdStr := strconv.Itoa(int(slotId))
|
||||||
|
cmdLine = append(cmdLine, []byte(slotIdStr))
|
||||||
|
}
|
||||||
|
targetNodeConn, err := cluster.connections.BorrowPeerClient(task.TargetNode)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
defer cluster.connections.ReturnPeerClient(targetNodeConn)
|
||||||
|
reply := targetNodeConn.Send(cmdLine)
|
||||||
|
if protocol.IsOKReply(reply) {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
return protocol.MakeErrReply("")
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cluster *Cluster) makeRebalancePlan() ([]*raft.MigratingTask, error) {
|
||||||
|
nodes, err := cluster.raftNode.GetNodes()
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
avgSlot := int(math.Ceil(float64(SlotCount) / float64(len(nodes))))
|
||||||
|
var migratings []*raft.MigratingTask
|
||||||
|
cluster.raftNode.FSM.WithReadLock(func(fsm *raft.FSM) {
|
||||||
|
var exportingNodes []string
|
||||||
|
var importingNodes []string
|
||||||
|
for _, node := range nodes {
|
||||||
|
nodeId := string(node.ID)
|
||||||
|
nodeSlots := fsm.Node2Slot[nodeId]
|
||||||
|
if len(nodeSlots) > avgSlot+1 {
|
||||||
|
exportingNodes = append(exportingNodes, nodeId)
|
||||||
|
}
|
||||||
|
if len(nodeSlots) < avgSlot-1 {
|
||||||
|
importingNodes = append(importingNodes, nodeId)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
importIndex := 0
|
||||||
|
exportIndex := 0
|
||||||
|
var exportSlots []uint32
|
||||||
|
for importIndex < len(importingNodes) && exportIndex < len(exportingNodes) {
|
||||||
|
exportNode := exportingNodes[exportIndex]
|
||||||
|
if len(exportSlots) == 0 {
|
||||||
|
exportNodeSlots := fsm.Node2Slot[exportNode]
|
||||||
|
exportCount := len(exportNodeSlots) - avgSlot
|
||||||
|
exportSlots = exportNodeSlots[0:exportCount]
|
||||||
|
}
|
||||||
|
importNode := importingNodes[importIndex]
|
||||||
|
importNodeCurrentIndex := fsm.Node2Slot[importNode]
|
||||||
|
requirements := avgSlot - len(importNodeCurrentIndex)
|
||||||
|
task := &raft.MigratingTask{
|
||||||
|
ID: utils.RandString(20),
|
||||||
|
SrcNode: exportNode,
|
||||||
|
TargetNode: importNode,
|
||||||
|
}
|
||||||
|
if requirements <= len(exportSlots) {
|
||||||
|
// exportSlots 可以提供足够的 slots, importingNode 处理完毕
|
||||||
|
task.Slots = exportSlots[0:requirements]
|
||||||
|
exportSlots = exportSlots[requirements:]
|
||||||
|
importIndex++
|
||||||
|
} else {
|
||||||
|
// exportSlots 无法提供足够的 slots, exportingNode 处理完毕
|
||||||
|
task.Slots = exportSlots
|
||||||
|
exportSlots = nil
|
||||||
|
exportIndex++
|
||||||
|
}
|
||||||
|
migratings = append(migratings, task)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
return migratings, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cluster *Cluster) waitCommitted(peer string, logIndex uint64) error {
|
||||||
|
srcNodeConn, err := cluster.connections.BorrowPeerClient(peer)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
defer cluster.connections.ReturnPeerClient(srcNodeConn)
|
||||||
|
var peerIndex uint64
|
||||||
|
for i := 0; i < 50; i++ {
|
||||||
|
reply := srcNodeConn.Send(utils.ToCmdLine(getCommittedIndexCommand))
|
||||||
|
switch reply := reply.(type) {
|
||||||
|
case *protocol.IntReply:
|
||||||
|
peerIndex = uint64(reply.Code)
|
||||||
|
if peerIndex >= logIndex {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
case *protocol.StandardErrReply:
|
||||||
|
logger.Infof("get committed index failed: %v", reply.Error())
|
||||||
|
default:
|
||||||
|
logger.Infof("get committed index unknown responseL %+v", reply.ToBytes())
|
||||||
|
}
|
||||||
|
time.Sleep(time.Millisecond * 100)
|
||||||
|
}
|
||||||
|
return errors.New("wait committed timeout")
|
||||||
|
}
|
||||||
|
|
||||||
|
// execMigrationChangeRoute should be exectued at leader
|
||||||
|
// it proposes EventFinishMigrate through raft, to change the route to the new node
|
||||||
|
// it returns until the proposal has been accepted by the majority and two related nodes
|
||||||
|
// format: cluster.migration.changeroute taskid
|
||||||
|
func execMigrationChangeRoute(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||||
|
if len(cmdLine) != 2 {
|
||||||
|
return protocol.MakeArgNumErrReply(joinClusterCommand)
|
||||||
|
}
|
||||||
|
state := cluster.raftNode.State()
|
||||||
|
if state != raft.Leader {
|
||||||
|
// I am not leader, forward request to leader
|
||||||
|
leaderConn, err := cluster.BorrowLeaderClient()
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply(err.Error())
|
||||||
|
}
|
||||||
|
defer cluster.connections.ReturnPeerClient(leaderConn)
|
||||||
|
return leaderConn.Send(cmdLine)
|
||||||
|
}
|
||||||
|
taskId := string(cmdLine[1])
|
||||||
|
logger.Infof("change route for migration %s", taskId)
|
||||||
|
task := cluster.raftNode.FSM.GetMigratingTask(taskId)
|
||||||
|
if task == nil {
|
||||||
|
return protocol.MakeErrReply("ERR task not found")
|
||||||
|
}
|
||||||
|
logger.Infof("change route for migration %s, got task info", taskId)
|
||||||
|
// propose
|
||||||
|
logIndex, err := cluster.raftNode.Propose(&raft.LogEntry{
|
||||||
|
Event: raft.EventFinishMigrate,
|
||||||
|
MigratingTask: task,
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply("ERR " + err.Error())
|
||||||
|
}
|
||||||
|
logger.Infof("change route for migration %s, raft proposed", taskId)
|
||||||
|
|
||||||
|
// confirm the 2 related node committed this log
|
||||||
|
err = cluster.waitCommitted(task.SrcNode, logIndex)
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply("ERR " + err.Error())
|
||||||
|
}
|
||||||
|
logger.Infof("change route for migration %s, confirm source node finished", taskId)
|
||||||
|
|
||||||
|
err = cluster.waitCommitted(task.TargetNode, logIndex)
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply("ERR " + err.Error())
|
||||||
|
}
|
||||||
|
logger.Infof("change route for migration %s, confirm target node finished", taskId)
|
||||||
|
|
||||||
|
return protocol.MakeOkReply()
|
||||||
|
}
|
303
cluster/core/migration.go
Normal file
303
cluster/core/migration.go
Normal file
@@ -0,0 +1,303 @@
|
|||||||
|
package core
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"strconv"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/hdt3213/godis/aof"
|
||||||
|
"github.com/hdt3213/godis/cluster/raft"
|
||||||
|
"github.com/hdt3213/godis/datastruct/set"
|
||||||
|
"github.com/hdt3213/godis/interface/database"
|
||||||
|
"github.com/hdt3213/godis/interface/redis"
|
||||||
|
"github.com/hdt3213/godis/lib/logger"
|
||||||
|
"github.com/hdt3213/godis/lib/utils"
|
||||||
|
"github.com/hdt3213/godis/redis/connection"
|
||||||
|
"github.com/hdt3213/godis/redis/protocol"
|
||||||
|
)
|
||||||
|
|
||||||
|
const exportCommand = "cluster.migration.export"
|
||||||
|
const migrationDoneCommand = "cluster.migration.done"
|
||||||
|
const startMigrationCommand = "cluster.migration.start"
|
||||||
|
|
||||||
|
func init() {
|
||||||
|
RegisterCmd(exportCommand, execExport)
|
||||||
|
RegisterCmd(migrationDoneCommand, execFinishExport)
|
||||||
|
RegisterCmd(startMigrationCommand, execStartMigration)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (sm *slotStatus) startExporting() protocol.ErrorReply {
|
||||||
|
sm.mu.Lock()
|
||||||
|
defer sm.mu.Unlock()
|
||||||
|
if sm.state != slotStateHosting {
|
||||||
|
return protocol.MakeErrReply("slot host is not hosting")
|
||||||
|
}
|
||||||
|
sm.state = slotStateExporting
|
||||||
|
sm.dirtyKeys = set.Make()
|
||||||
|
sm.exportSnapshot = sm.keys.ShallowCopy()
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (sm *slotStatus) finishExportingWithinLock() {
|
||||||
|
sm.state = slotStateHosting
|
||||||
|
sm.dirtyKeys = nil
|
||||||
|
sm.exportSnapshot = nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cluster *Cluster) injectInsertCallback() {
|
||||||
|
cb := func(dbIndex int, key string, entity *database.DataEntity) {
|
||||||
|
slotIndex := GetSlot(key)
|
||||||
|
slotManager := cluster.slotsManager.getSlot(slotIndex)
|
||||||
|
slotManager.mu.Lock()
|
||||||
|
defer slotManager.mu.Unlock()
|
||||||
|
slotManager.keys.Add(key)
|
||||||
|
if slotManager.state == slotStateExporting {
|
||||||
|
slotManager.dirtyKeys.Add(key)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
cluster.db.SetKeyInsertedCallback(cb)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cluster *Cluster) injectDeleteCallback() {
|
||||||
|
cb := func(dbIndex int, key string, entity *database.DataEntity) {
|
||||||
|
slotIndex := GetSlot(key)
|
||||||
|
slotManager := cluster.slotsManager.getSlot(slotIndex)
|
||||||
|
slotManager.mu.Lock()
|
||||||
|
defer slotManager.mu.Unlock()
|
||||||
|
slotManager.keys.Remove(key)
|
||||||
|
if slotManager.state == slotStateExporting {
|
||||||
|
// 可能数据迁移后再进行了一次 delete, 所以这也是一个 dirty key
|
||||||
|
slotManager.dirtyKeys.Add(key)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
cluster.db.SetKeyDeletedCallback(cb)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cluster *Cluster) dumpDataThroughConnection(c redis.Connection, keyset *set.Set) {
|
||||||
|
keyset.ForEach(func(key string) bool {
|
||||||
|
entity, ok := cluster.db.GetEntity(0, key)
|
||||||
|
if ok {
|
||||||
|
ret := aof.EntityToCmd(key, entity)
|
||||||
|
// todo: handle error and close connection
|
||||||
|
_, _ = c.Write(ret.ToBytes())
|
||||||
|
expire := cluster.db.GetExpiration(0, key)
|
||||||
|
if expire != nil {
|
||||||
|
ret = aof.MakeExpireCmd(key, *expire)
|
||||||
|
_, _ = c.Write(ret.ToBytes())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
// execExport dump slots data to caller
|
||||||
|
// command line: cluster.export taskId
|
||||||
|
func execExport(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||||
|
if len(cmdLine) != 2 {
|
||||||
|
return protocol.MakeArgNumErrReply(exportCommand)
|
||||||
|
}
|
||||||
|
|
||||||
|
var task *raft.MigratingTask
|
||||||
|
taskId := string(cmdLine[1])
|
||||||
|
for i := 0; i < 50; i++ {
|
||||||
|
task = cluster.raftNode.FSM.GetMigratingTask(taskId)
|
||||||
|
if task == nil {
|
||||||
|
time.Sleep(time.Millisecond * 100)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if task == nil {
|
||||||
|
return protocol.MakeErrReply("ERR get migrating task timeout")
|
||||||
|
}
|
||||||
|
cluster.slotsManager.mu.Lock()
|
||||||
|
if cluster.slotsManager.importingTask != nil {
|
||||||
|
cluster.slotsManager.mu.Unlock()
|
||||||
|
return protocol.MakeErrReply("ERR another migrating task in progress")
|
||||||
|
}
|
||||||
|
cluster.slotsManager.importingTask = task
|
||||||
|
cluster.slotsManager.mu.Unlock()
|
||||||
|
|
||||||
|
for _, slotId := range task.Slots {
|
||||||
|
slotManager := cluster.slotsManager.getSlot(slotId)
|
||||||
|
errReply := slotManager.startExporting()
|
||||||
|
if errReply != nil {
|
||||||
|
return errReply
|
||||||
|
}
|
||||||
|
cluster.dumpDataThroughConnection(c, slotManager.exportSnapshot)
|
||||||
|
logger.Info("finish dump slot ", slotId)
|
||||||
|
// send a ok reply to tell requesting node dump finished
|
||||||
|
}
|
||||||
|
c.Write(protocol.MakeOkReply().ToBytes())
|
||||||
|
return &protocol.NoReply{}
|
||||||
|
}
|
||||||
|
|
||||||
|
// execFinishExport
|
||||||
|
// command line: migrationDoneCommand taskId
|
||||||
|
// firstly dump dirty data from connection `c`, then returns an OK response
|
||||||
|
func execFinishExport(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||||
|
if len(cmdLine) != 2 {
|
||||||
|
return protocol.MakeArgNumErrReply(exportCommand)
|
||||||
|
}
|
||||||
|
// get MigratingTask from raft
|
||||||
|
var task *raft.MigratingTask
|
||||||
|
taskId := string(cmdLine[1])
|
||||||
|
logger.Info("finishing migration task: " + taskId)
|
||||||
|
for i := 0; i < 50; i++ {
|
||||||
|
task = cluster.raftNode.FSM.GetMigratingTask(taskId)
|
||||||
|
if task == nil {
|
||||||
|
time.Sleep(time.Millisecond * 100)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if task == nil {
|
||||||
|
return protocol.MakeErrReply("ERR get migrating task timeout")
|
||||||
|
}
|
||||||
|
logger.Infof("finishing migration task %s, got task info", taskId)
|
||||||
|
|
||||||
|
|
||||||
|
// transport dirty keys within lock, lock will be released while migration done
|
||||||
|
var lockedSlots []uint32
|
||||||
|
defer func() {
|
||||||
|
for i := len(lockedSlots) - 1; i >= 0; i-- {
|
||||||
|
slotId := lockedSlots[i]
|
||||||
|
slotManager := cluster.slotsManager.getSlot(slotId)
|
||||||
|
slotManager.mu.Unlock()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
for _, slotId := range task.Slots {
|
||||||
|
slotManager := cluster.slotsManager.getSlot(slotId)
|
||||||
|
slotManager.mu.Lock()
|
||||||
|
lockedSlots = append(lockedSlots, slotId)
|
||||||
|
cluster.dumpDataThroughConnection(c, slotManager.dirtyKeys)
|
||||||
|
slotManager.finishExportingWithinLock()
|
||||||
|
}
|
||||||
|
logger.Infof("finishing migration task %s, dirty keys transported", taskId)
|
||||||
|
|
||||||
|
|
||||||
|
// propose migrate finish
|
||||||
|
leaderConn, err := cluster.BorrowLeaderClient()
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply(err.Error())
|
||||||
|
}
|
||||||
|
defer cluster.connections.ReturnPeerClient(leaderConn)
|
||||||
|
reply := leaderConn.Send(utils.ToCmdLine(migrationChangeRouteCommand, taskId))
|
||||||
|
switch reply := reply.(type) {
|
||||||
|
case *protocol.StatusReply, *protocol.OkReply:
|
||||||
|
return protocol.MakeOkReply()
|
||||||
|
case *protocol.StandardErrReply:
|
||||||
|
logger.Infof("migration done command failed: %v", reply.Error())
|
||||||
|
default:
|
||||||
|
logger.Infof("finish migration request unknown response %s", string(reply.ToBytes()))
|
||||||
|
}
|
||||||
|
logger.Infof("finishing migration task %s, route changed", taskId)
|
||||||
|
|
||||||
|
c.Write(protocol.MakeOkReply().ToBytes())
|
||||||
|
return &protocol.NoReply{}
|
||||||
|
}
|
||||||
|
|
||||||
|
// execStartMigration receives startMigrationCommand from leader and start migration job at background
|
||||||
|
// command line: startMigrationCommand taskId srcNode slotId1 [slotId2]...
|
||||||
|
func execStartMigration(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||||
|
if len(cmdLine) < 4 {
|
||||||
|
return protocol.MakeArgNumErrReply(startMigrationCommand)
|
||||||
|
}
|
||||||
|
taskId := string(cmdLine[1])
|
||||||
|
srcNode := string(cmdLine[2])
|
||||||
|
var slotIds []uint32
|
||||||
|
for _, slotIdStr := range cmdLine[3:] {
|
||||||
|
slotId, err := strconv.Atoi(string(slotIdStr))
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply("illegal slot id: " + string(slotIdStr))
|
||||||
|
}
|
||||||
|
slotIds = append(slotIds, uint32(slotId))
|
||||||
|
}
|
||||||
|
task := &raft.MigratingTask{
|
||||||
|
ID: taskId,
|
||||||
|
SrcNode: srcNode,
|
||||||
|
TargetNode: cluster.SelfID(),
|
||||||
|
Slots: slotIds,
|
||||||
|
}
|
||||||
|
cluster.slotsManager.mu.Lock()
|
||||||
|
cluster.slotsManager.importingTask = task
|
||||||
|
cluster.slotsManager.mu.Unlock()
|
||||||
|
logger.Infof("received importing task %s, %d slots to import", task.ID, len(task.Slots))
|
||||||
|
go func() {
|
||||||
|
defer func() {
|
||||||
|
if e := recover(); e != nil {
|
||||||
|
logger.Errorf("panic: %v", e)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
cluster.doImports(task)
|
||||||
|
}()
|
||||||
|
return protocol.MakeOkReply()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cluster *Cluster) doImports(task *raft.MigratingTask) error {
|
||||||
|
/// STEP1: export
|
||||||
|
cmdLine := utils.ToCmdLine(exportCommand, task.ID)
|
||||||
|
stream, err := cluster.connections.NewStream(task.SrcNode, cmdLine)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
defer stream.Close()
|
||||||
|
|
||||||
|
fakeConn := connection.NewFakeConn()
|
||||||
|
|
||||||
|
// todo: import 状态的 slots 只接受 srcNode 的写入
|
||||||
|
recvLoop:
|
||||||
|
for proto := range stream.Stream() {
|
||||||
|
if proto.Err != nil {
|
||||||
|
return fmt.Errorf("export error: %v", err)
|
||||||
|
}
|
||||||
|
switch reply := proto.Data.(type) {
|
||||||
|
case *protocol.MultiBulkReply:
|
||||||
|
_ = cluster.db.Exec(fakeConn, reply.Args)
|
||||||
|
case *protocol.StatusReply, *protocol.OkReply:
|
||||||
|
if protocol.IsOKReply(reply) {
|
||||||
|
logger.Info("importing task received OK reply, phase 1 done")
|
||||||
|
break recvLoop
|
||||||
|
} else {
|
||||||
|
msg := fmt.Sprintf("migrate error: %s", string(reply.ToBytes()))
|
||||||
|
logger.Errorf(msg)
|
||||||
|
return protocol.MakeErrReply(msg)
|
||||||
|
}
|
||||||
|
case protocol.ErrorReply:
|
||||||
|
// todo: return slot to former host node
|
||||||
|
msg := fmt.Sprintf("migrate error: %s", reply.Error())
|
||||||
|
logger.Errorf(msg)
|
||||||
|
return protocol.MakeErrReply(msg)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
///STEP3: 通知 srcNode 进入结束流程
|
||||||
|
stream2, err := cluster.connections.NewStream(task.SrcNode, utils.ToCmdLine(migrationDoneCommand, task.ID))
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
defer stream2.Close()
|
||||||
|
// receive dirty datas
|
||||||
|
recvLoop2:
|
||||||
|
for proto := range stream2.Stream() {
|
||||||
|
if proto.Err != nil {
|
||||||
|
return fmt.Errorf("export error: %v", err)
|
||||||
|
}
|
||||||
|
switch reply := proto.Data.(type) {
|
||||||
|
case *protocol.MultiBulkReply:
|
||||||
|
_ = cluster.db.Exec(fakeConn, reply.Args)
|
||||||
|
case *protocol.StatusReply, *protocol.OkReply:
|
||||||
|
if protocol.IsOKReply(reply) {
|
||||||
|
logger.Info("importing task received OK reply, phase 2 done")
|
||||||
|
break recvLoop2
|
||||||
|
} else {
|
||||||
|
msg := fmt.Sprintf("migrate error: %s", string(reply.ToBytes()))
|
||||||
|
logger.Errorf(msg)
|
||||||
|
return protocol.MakeErrReply(msg)
|
||||||
|
}
|
||||||
|
case protocol.ErrorReply:
|
||||||
|
// todo: return slot to former host node
|
||||||
|
msg := fmt.Sprintf("migrate error: %s", reply.Error())
|
||||||
|
logger.Errorf(msg)
|
||||||
|
return protocol.MakeErrReply(msg)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
68
cluster/core/utils.go
Normal file
68
cluster/core/utils.go
Normal file
@@ -0,0 +1,68 @@
|
|||||||
|
package core
|
||||||
|
|
||||||
|
import (
|
||||||
|
"hash/crc32"
|
||||||
|
"strings"
|
||||||
|
|
||||||
|
"github.com/hdt3213/godis/interface/redis"
|
||||||
|
"github.com/hdt3213/godis/redis/protocol"
|
||||||
|
)
|
||||||
|
|
||||||
|
const SlotCount int = 1024
|
||||||
|
|
||||||
|
const getCommittedIndexCommand = "raft.committedindex"
|
||||||
|
|
||||||
|
func init() {
|
||||||
|
RegisterCmd(getCommittedIndexCommand, execRaftCommittedIndex)
|
||||||
|
}
|
||||||
|
|
||||||
|
// relay function relays command to peer or calls cluster.Exec
|
||||||
|
func (cluster *Cluster) Relay(peerId string, c redis.Connection, cmdLine [][]byte) redis.Reply {
|
||||||
|
// use a variable to allow injecting stub for testing, see defaultRelayImpl
|
||||||
|
if peerId == cluster.SelfID() {
|
||||||
|
// to self db
|
||||||
|
return cluster.Exec(c, cmdLine)
|
||||||
|
}
|
||||||
|
// peerId is peer.Addr
|
||||||
|
cli, err := cluster.connections.BorrowPeerClient(peerId)
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply(err.Error())
|
||||||
|
}
|
||||||
|
defer func() {
|
||||||
|
_ = cluster.connections.ReturnPeerClient(cli)
|
||||||
|
}()
|
||||||
|
return cli.Send(cmdLine)
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetPartitionKey extract hashtag
|
||||||
|
func GetPartitionKey(key string) string {
|
||||||
|
beg := strings.Index(key, "{")
|
||||||
|
if beg == -1 {
|
||||||
|
return key
|
||||||
|
}
|
||||||
|
end := strings.Index(key, "}")
|
||||||
|
if end == -1 || end == beg+1 {
|
||||||
|
return key
|
||||||
|
}
|
||||||
|
return key[beg+1 : end]
|
||||||
|
}
|
||||||
|
|
||||||
|
func GetSlot(key string) uint32 {
|
||||||
|
partitionKey := GetPartitionKey(key)
|
||||||
|
return crc32.ChecksumIEEE([]byte(partitionKey)) % uint32(SlotCount)
|
||||||
|
}
|
||||||
|
|
||||||
|
// pickNode returns the node id hosting the given slot.
|
||||||
|
// If the slot is migrating, return the node which is exporting the slot
|
||||||
|
func (cluster *Cluster) PickNode(slotID uint32) string {
|
||||||
|
return cluster.raftNode.FSM.PickNode(slotID)
|
||||||
|
}
|
||||||
|
|
||||||
|
// format: raft.committedindex
|
||||||
|
func execRaftCommittedIndex(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||||
|
index, err := cluster.raftNode.CommittedIndex()
|
||||||
|
if err != nil {
|
||||||
|
return protocol.MakeErrReply(err.Error())
|
||||||
|
}
|
||||||
|
return protocol.MakeIntReply(int64(index))
|
||||||
|
}
|
203
cluster/raft/fsm.go
Normal file
203
cluster/raft/fsm.go
Normal file
@@ -0,0 +1,203 @@
|
|||||||
|
package raft
|
||||||
|
|
||||||
|
import (
|
||||||
|
"encoding/json"
|
||||||
|
"io"
|
||||||
|
"sort"
|
||||||
|
"sync"
|
||||||
|
|
||||||
|
"github.com/hashicorp/raft"
|
||||||
|
)
|
||||||
|
|
||||||
|
// FSM implements raft.FSM
|
||||||
|
// It stores node-slots mapping to providing routing service
|
||||||
|
//
|
||||||
|
// A request pointing to a migrating slot will be routed to source node.
|
||||||
|
// After the command being executed, the updates will be relayed to the target node
|
||||||
|
//
|
||||||
|
// If the target node crashes during migrating, the migration will be canceled.
|
||||||
|
// All related commands will be routed to the source node
|
||||||
|
type FSM struct {
|
||||||
|
mu sync.RWMutex
|
||||||
|
Node2Slot map[string][]uint32 // nodeID -> slotIDs, slotIDs is in ascending order and distinct
|
||||||
|
Slot2Node map[uint32]string // slotID -> nodeID
|
||||||
|
Migratings map[string]*MigratingTask // taskId -> task
|
||||||
|
}
|
||||||
|
|
||||||
|
// MigratingTask
|
||||||
|
// It is immutable
|
||||||
|
type MigratingTask struct {
|
||||||
|
ID string
|
||||||
|
SrcNode string
|
||||||
|
TargetNode string
|
||||||
|
|
||||||
|
// Slots stores slots to migrate in this event
|
||||||
|
Slots []uint32
|
||||||
|
}
|
||||||
|
|
||||||
|
// InitTask
|
||||||
|
type InitTask struct {
|
||||||
|
Leader string
|
||||||
|
SlotCount int
|
||||||
|
}
|
||||||
|
|
||||||
|
// implements FSM.Apply after you created a new raft event
|
||||||
|
const (
|
||||||
|
EventStartMigrate = iota + 1
|
||||||
|
EventFinishMigrate
|
||||||
|
EventSeedStart
|
||||||
|
)
|
||||||
|
|
||||||
|
// LogEntry is an entry in raft log, stores a change of cluster
|
||||||
|
type LogEntry struct {
|
||||||
|
Event int
|
||||||
|
MigratingTask *MigratingTask `json:"MigratingTask,omitempty"`
|
||||||
|
InitTask *InitTask
|
||||||
|
}
|
||||||
|
|
||||||
|
// Apply is called once a log entry is committed by a majority of the cluster.
|
||||||
|
func (fsm *FSM) Apply(log *raft.Log) interface{} {
|
||||||
|
fsm.mu.Lock()
|
||||||
|
defer fsm.mu.Unlock()
|
||||||
|
|
||||||
|
entry := &LogEntry{}
|
||||||
|
err := json.Unmarshal(log.Data, entry)
|
||||||
|
if err != nil {
|
||||||
|
panic(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
/// event handler
|
||||||
|
if entry.Event == EventStartMigrate {
|
||||||
|
task := entry.MigratingTask
|
||||||
|
fsm.Migratings[task.ID] = task
|
||||||
|
} else if entry.Event == EventFinishMigrate {
|
||||||
|
task := entry.MigratingTask
|
||||||
|
delete(fsm.Migratings, task.ID)
|
||||||
|
fsm.addSlots(task.TargetNode, task.Slots)
|
||||||
|
fsm.removeSlots(task.SrcNode, task.Slots)
|
||||||
|
} else if entry.Event == EventSeedStart {
|
||||||
|
slots := make([]uint32, int(entry.InitTask.SlotCount))
|
||||||
|
for i := 0; i < entry.InitTask.SlotCount; i++ {
|
||||||
|
fsm.Slot2Node[uint32(i)] = entry.InitTask.Leader
|
||||||
|
slots[i] = uint32(i)
|
||||||
|
}
|
||||||
|
fsm.Node2Slot[entry.InitTask.Leader] = slots
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (fsm *FSM) addSlots(nodeID string, slots []uint32) {
|
||||||
|
for _, slotId := range slots {
|
||||||
|
/// update node2Slot
|
||||||
|
index := sort.Search(len(fsm.Node2Slot[nodeID]), func(i int) bool {
|
||||||
|
return fsm.Node2Slot[nodeID][i] >= slotId
|
||||||
|
})
|
||||||
|
if !(index < len(fsm.Node2Slot[nodeID]) && fsm.Node2Slot[nodeID][index] == slotId) {
|
||||||
|
// not found in node's slots, insert
|
||||||
|
fsm.Node2Slot[nodeID] = append(fsm.Node2Slot[nodeID][:index],
|
||||||
|
append([]uint32{slotId}, fsm.Node2Slot[nodeID][index:]...)...)
|
||||||
|
}
|
||||||
|
/// update slot2Node
|
||||||
|
fsm.Slot2Node[slotId] = nodeID
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (fsm *FSM) removeSlots(nodeID string, slots []uint32) {
|
||||||
|
for _, slotId := range slots {
|
||||||
|
/// update node2slot
|
||||||
|
index := sort.Search(len(fsm.Node2Slot[nodeID]), func(i int) bool { return fsm.Node2Slot[nodeID][i] >= slotId })
|
||||||
|
// found slot remove
|
||||||
|
for index < len(fsm.Node2Slot[nodeID]) && fsm.Node2Slot[nodeID][index] == slotId {
|
||||||
|
fsm.Node2Slot[nodeID] = append(fsm.Node2Slot[nodeID][:index], fsm.Node2Slot[nodeID][index+1:]...)
|
||||||
|
}
|
||||||
|
// update slot2node
|
||||||
|
if fsm.Slot2Node[slotId] == nodeID {
|
||||||
|
delete(fsm.Slot2Node, slotId)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (fsm *FSM) GetMigratingTask(taskId string) *MigratingTask {
|
||||||
|
fsm.mu.RLock()
|
||||||
|
defer fsm.mu.RUnlock()
|
||||||
|
return fsm.Migratings[taskId]
|
||||||
|
}
|
||||||
|
|
||||||
|
// FSMSnapshot stores necessary data to restore FSM
|
||||||
|
type FSMSnapshot struct {
|
||||||
|
Slot2Node map[uint32]string // slotID -> nodeID
|
||||||
|
Migratings map[string]*MigratingTask
|
||||||
|
}
|
||||||
|
|
||||||
|
func (snapshot *FSMSnapshot) Persist(sink raft.SnapshotSink) error {
|
||||||
|
err := func() error {
|
||||||
|
data, err := json.Marshal(snapshot)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
_, err = sink.Write(data)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
return sink.Close()
|
||||||
|
}()
|
||||||
|
if err != nil {
|
||||||
|
sink.Cancel()
|
||||||
|
}
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
func (snapshot *FSMSnapshot) Release() {}
|
||||||
|
|
||||||
|
func (fsm *FSM) Snapshot() (raft.FSMSnapshot, error) {
|
||||||
|
fsm.mu.RLock()
|
||||||
|
defer fsm.mu.RUnlock()
|
||||||
|
slot2Node := make(map[uint32]string)
|
||||||
|
for k, v := range fsm.Slot2Node {
|
||||||
|
slot2Node[k] = v
|
||||||
|
}
|
||||||
|
migratings := make(map[string]*MigratingTask)
|
||||||
|
for k, v := range fsm.Migratings {
|
||||||
|
migratings[k] = v
|
||||||
|
}
|
||||||
|
return &FSMSnapshot{
|
||||||
|
Slot2Node: slot2Node,
|
||||||
|
Migratings: migratings,
|
||||||
|
}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (fsm *FSM) Restore(src io.ReadCloser) error {
|
||||||
|
fsm.mu.Lock()
|
||||||
|
defer fsm.mu.Unlock()
|
||||||
|
data, err := io.ReadAll(src)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
snapshot := &FSMSnapshot{}
|
||||||
|
err = json.Unmarshal(data, snapshot)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
fsm.Slot2Node = snapshot.Slot2Node
|
||||||
|
fsm.Migratings = snapshot.Migratings
|
||||||
|
fsm.Node2Slot = make(map[string][]uint32)
|
||||||
|
for slot, node := range snapshot.Slot2Node {
|
||||||
|
fsm.Node2Slot[node] = append(fsm.Node2Slot[node], slot)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// PickNode returns node hosting slot, ignore migrating
|
||||||
|
func (fsm *FSM) PickNode(slot uint32) string {
|
||||||
|
fsm.mu.RLock()
|
||||||
|
defer fsm.mu.RUnlock()
|
||||||
|
return fsm.Slot2Node[slot]
|
||||||
|
}
|
||||||
|
|
||||||
|
// WithReadLock allow invoker do something complicated with read lock
|
||||||
|
func (fsm *FSM) WithReadLock(fn func(fsm *FSM)) {
|
||||||
|
fsm.mu.RLock()
|
||||||
|
defer fsm.mu.RUnlock()
|
||||||
|
fn(fsm)
|
||||||
|
}
|
189
cluster/raft/raft.go
Normal file
189
cluster/raft/raft.go
Normal file
@@ -0,0 +1,189 @@
|
|||||||
|
package raft
|
||||||
|
|
||||||
|
import (
|
||||||
|
"encoding/json"
|
||||||
|
"errors"
|
||||||
|
"fmt"
|
||||||
|
"net"
|
||||||
|
"os"
|
||||||
|
"path/filepath"
|
||||||
|
"strconv"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/hashicorp/raft"
|
||||||
|
raftboltdb "github.com/hashicorp/raft-boltdb"
|
||||||
|
)
|
||||||
|
|
||||||
|
type Node struct {
|
||||||
|
Cfg *RaftConfig
|
||||||
|
inner *raft.Raft
|
||||||
|
FSM *FSM
|
||||||
|
logStore raft.LogStore
|
||||||
|
stableStore raft.StableStore
|
||||||
|
snapshotStore raft.SnapshotStore
|
||||||
|
transport raft.Transport
|
||||||
|
}
|
||||||
|
|
||||||
|
type RaftConfig struct {
|
||||||
|
RedisAdvertiseAddr string // it also be used as node id,
|
||||||
|
RaftListenAddr string
|
||||||
|
RaftAdvertiseAddr string
|
||||||
|
Dir string
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cfg *RaftConfig) ID() string {
|
||||||
|
return cfg.RedisAdvertiseAddr
|
||||||
|
}
|
||||||
|
|
||||||
|
var Leader = raft.Leader
|
||||||
|
var Follower = raft.Follower
|
||||||
|
var Candidate = raft.Candidate
|
||||||
|
|
||||||
|
func StartNode(cfg *RaftConfig) (*Node, error) {
|
||||||
|
if cfg.RaftAdvertiseAddr == "" {
|
||||||
|
cfg.RaftAdvertiseAddr = cfg.RaftListenAddr
|
||||||
|
}
|
||||||
|
config := raft.DefaultConfig()
|
||||||
|
config.LocalID = raft.ServerID(cfg.ID())
|
||||||
|
if config.LocalID == "" {
|
||||||
|
// cfg.ID() actually is cfg.RedisAdvertiseAddr
|
||||||
|
return nil, errors.New("redis address is required")
|
||||||
|
}
|
||||||
|
leaderNotifyCh := make(chan bool, 10)
|
||||||
|
config.NotifyCh = leaderNotifyCh
|
||||||
|
|
||||||
|
addr, err := net.ResolveTCPAddr("tcp", cfg.RaftAdvertiseAddr)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
transport, err := raft.NewTCPTransport(cfg.RaftListenAddr, addr, 3, 10*time.Second, os.Stderr)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
// todo: mkdir if possible
|
||||||
|
snapshotStore, err := raft.NewFileSnapshotStore(cfg.Dir, 2, os.Stderr)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
boltDB, err := raftboltdb.New(raftboltdb.Options{
|
||||||
|
Path: filepath.Join(cfg.Dir, "raft.db"),
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
storage := &FSM{
|
||||||
|
Node2Slot: make(map[string][]uint32),
|
||||||
|
Slot2Node: make(map[uint32]string),
|
||||||
|
Migratings: make(map[string]*MigratingTask),
|
||||||
|
}
|
||||||
|
|
||||||
|
logStore := boltDB
|
||||||
|
stableStore := boltDB
|
||||||
|
inner, err := raft.NewRaft(config, storage, logStore, stableStore, snapshotStore, transport)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
return &Node{
|
||||||
|
Cfg: cfg,
|
||||||
|
inner: inner,
|
||||||
|
FSM: storage,
|
||||||
|
logStore: logStore,
|
||||||
|
stableStore: stableStore,
|
||||||
|
snapshotStore: snapshotStore,
|
||||||
|
transport: transport,
|
||||||
|
}, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (node *Node) HasExistingState() (bool, error) {
|
||||||
|
return raft.HasExistingState(node.logStore, node.stableStore, node.snapshotStore)
|
||||||
|
}
|
||||||
|
|
||||||
|
// BootstrapCluster creates a raft cluster, and returns after self makes leader
|
||||||
|
func (node *Node) BootstrapCluster(slotCount int) error {
|
||||||
|
future := node.inner.BootstrapCluster(raft.Configuration{
|
||||||
|
Servers: []raft.Server{
|
||||||
|
{
|
||||||
|
ID: raft.ServerID(node.Cfg.ID()),
|
||||||
|
Address: node.transport.LocalAddr(),
|
||||||
|
},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
err := future.Error()
|
||||||
|
if err != nil {
|
||||||
|
return fmt.Errorf("BootstrapCluster failed: %v", err)
|
||||||
|
}
|
||||||
|
// wait self leader
|
||||||
|
for {
|
||||||
|
if node.State() == raft.Leader {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
time.Sleep(100 * time.Millisecond)
|
||||||
|
}
|
||||||
|
// init fsm
|
||||||
|
_, err = node.Propose(&LogEntry{Event: EventSeedStart, InitTask: &InitTask{
|
||||||
|
Leader: node.Cfg.ID(),
|
||||||
|
SlotCount: slotCount,
|
||||||
|
}})
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
func (node *Node) Shutdown() error {
|
||||||
|
future := node.inner.Shutdown()
|
||||||
|
return future.Error()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (node *Node) State() raft.RaftState {
|
||||||
|
return node.inner.State()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (node *Node) CommittedIndex() (uint64, error) {
|
||||||
|
stats := node.inner.Stats()
|
||||||
|
committedIndex0 := stats["commit_index"]
|
||||||
|
return strconv.ParseUint(committedIndex0, 10, 64)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (node *Node) GetLeaderRedisAddress() string {
|
||||||
|
// redis advertise address used as leader id
|
||||||
|
_, id := node.inner.LeaderWithID()
|
||||||
|
return string(id)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (node *Node) GetNodes() ([]raft.Server, error) {
|
||||||
|
configFuture := node.inner.GetConfiguration()
|
||||||
|
if err := configFuture.Error(); err != nil {
|
||||||
|
return nil, fmt.Errorf("failed to get raft configuration: %v", err)
|
||||||
|
}
|
||||||
|
return configFuture.Configuration().Servers, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// HandleJoin handles join request, node must be leader
|
||||||
|
func (node *Node) HandleJoin(redisAddr, raftAddr string) error {
|
||||||
|
configFuture := node.inner.GetConfiguration()
|
||||||
|
if err := configFuture.Error(); err != nil {
|
||||||
|
return fmt.Errorf("failed to get raft configuration: %v", err)
|
||||||
|
}
|
||||||
|
id := raft.ServerID(redisAddr)
|
||||||
|
for _, srv := range configFuture.Configuration().Servers {
|
||||||
|
if srv.ID == id {
|
||||||
|
return errors.New("already in cluster")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
future := node.inner.AddVoter(id, raft.ServerAddress(raftAddr), 0, 0)
|
||||||
|
return future.Error()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (node *Node) Propose(event *LogEntry) (uint64, error) {
|
||||||
|
bin, err := json.Marshal(event)
|
||||||
|
if err != nil {
|
||||||
|
return 0, fmt.Errorf("marshal event failed: %v", err)
|
||||||
|
}
|
||||||
|
future := node.inner.Apply(bin, 0)
|
||||||
|
err = future.Error()
|
||||||
|
if err != nil {
|
||||||
|
return 0, fmt.Errorf("raft propose failed: %v", err)
|
||||||
|
}
|
||||||
|
return future.Index(), nil
|
||||||
|
}
|
@@ -40,15 +40,10 @@ type ServerProperties struct {
|
|||||||
SlaveAnnouncePort int `cfg:"slave-announce-port"`
|
SlaveAnnouncePort int `cfg:"slave-announce-port"`
|
||||||
SlaveAnnounceIP string `cfg:"slave-announce-ip"`
|
SlaveAnnounceIP string `cfg:"slave-announce-ip"`
|
||||||
ReplTimeout int `cfg:"repl-timeout"`
|
ReplTimeout int `cfg:"repl-timeout"`
|
||||||
|
|
||||||
ClusterEnable bool `cfg:"cluster-enable"`
|
ClusterEnable bool `cfg:"cluster-enable"`
|
||||||
ClusterAsSeed bool `cfg:"cluster-as-seed"`
|
ClusterAsSeed bool `cfg:"cluster-as-seed"`
|
||||||
ClusterSeed string `cfg:"cluster-seed"`
|
ClusterSeed string `cfg:"cluster-seed"`
|
||||||
ClusterConfigFile string `cfg:"cluster-config-file"`
|
|
||||||
|
|
||||||
// for cluster mode configuration
|
|
||||||
ClusterEnabled string `cfg:"cluster-enabled"` // Not used at present.
|
|
||||||
Peers []string `cfg:"peers"`
|
|
||||||
Self string `cfg:"self"`
|
|
||||||
|
|
||||||
// config file path
|
// config file path
|
||||||
CfPath string `cfg:"cf,omitempty"`
|
CfPath string `cfg:"cf,omitempty"`
|
||||||
|
@@ -164,7 +164,7 @@ func GenGodisInfoString(section string, db *Server) []byte {
|
|||||||
|
|
||||||
// getGodisRunningMode return godis running mode
|
// getGodisRunningMode return godis running mode
|
||||||
func getGodisRunningMode() string {
|
func getGodisRunningMode() string {
|
||||||
if config.Properties.ClusterEnabled == "yes" {
|
if config.Properties.ClusterEnable {
|
||||||
return config.ClusterMode
|
return config.ClusterMode
|
||||||
} else {
|
} else {
|
||||||
return config.StandaloneMode
|
return config.StandaloneMode
|
||||||
|
@@ -40,6 +40,18 @@ func computeCapacity(param int) (size int) {
|
|||||||
|
|
||||||
// MakeConcurrent creates ConcurrentDict with the given shard count
|
// MakeConcurrent creates ConcurrentDict with the given shard count
|
||||||
func MakeConcurrent(shardCount int) *ConcurrentDict {
|
func MakeConcurrent(shardCount int) *ConcurrentDict {
|
||||||
|
if shardCount == 1 {
|
||||||
|
table := []*shard{
|
||||||
|
{
|
||||||
|
m: make(map[string]interface{}),
|
||||||
|
},
|
||||||
|
}
|
||||||
|
return &ConcurrentDict{
|
||||||
|
count: 0,
|
||||||
|
table: table,
|
||||||
|
shardCount: shardCount,
|
||||||
|
}
|
||||||
|
}
|
||||||
shardCount = computeCapacity(shardCount)
|
shardCount = computeCapacity(shardCount)
|
||||||
table := make([]*shard, shardCount)
|
table := make([]*shard, shardCount)
|
||||||
for i := 0; i < shardCount; i++ {
|
for i := 0; i < shardCount; i++ {
|
||||||
@@ -66,10 +78,14 @@ func fnv32(key string) uint32 {
|
|||||||
return hash
|
return hash
|
||||||
}
|
}
|
||||||
|
|
||||||
func (dict *ConcurrentDict) spread(hashCode uint32) uint32 {
|
func (dict *ConcurrentDict) spread(key string) uint32 {
|
||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
|
if len(dict.table) == 1 {
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
hashCode := fnv32(key)
|
||||||
tableSize := uint32(len(dict.table))
|
tableSize := uint32(len(dict.table))
|
||||||
return (tableSize - 1) & hashCode
|
return (tableSize - 1) & hashCode
|
||||||
}
|
}
|
||||||
@@ -86,8 +102,7 @@ func (dict *ConcurrentDict) Get(key string) (val interface{}, exists bool) {
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
s.mutex.Lock()
|
s.mutex.Lock()
|
||||||
defer s.mutex.Unlock()
|
defer s.mutex.Unlock()
|
||||||
@@ -99,8 +114,7 @@ func (dict *ConcurrentDict) GetWithLock(key string) (val interface{}, exists boo
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
val, exists = s.m[key]
|
val, exists = s.m[key]
|
||||||
return
|
return
|
||||||
@@ -119,8 +133,7 @@ func (dict *ConcurrentDict) Put(key string, val interface{}) (result int) {
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
s.mutex.Lock()
|
s.mutex.Lock()
|
||||||
defer s.mutex.Unlock()
|
defer s.mutex.Unlock()
|
||||||
@@ -138,8 +151,7 @@ func (dict *ConcurrentDict) PutWithLock(key string, val interface{}) (result int
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
|
|
||||||
if _, ok := s.m[key]; ok {
|
if _, ok := s.m[key]; ok {
|
||||||
@@ -156,8 +168,7 @@ func (dict *ConcurrentDict) PutIfAbsent(key string, val interface{}) (result int
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
s.mutex.Lock()
|
s.mutex.Lock()
|
||||||
defer s.mutex.Unlock()
|
defer s.mutex.Unlock()
|
||||||
@@ -174,8 +185,7 @@ func (dict *ConcurrentDict) PutIfAbsentWithLock(key string, val interface{}) (re
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
|
|
||||||
if _, ok := s.m[key]; ok {
|
if _, ok := s.m[key]; ok {
|
||||||
@@ -191,8 +201,7 @@ func (dict *ConcurrentDict) PutIfExists(key string, val interface{}) (result int
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
s.mutex.Lock()
|
s.mutex.Lock()
|
||||||
defer s.mutex.Unlock()
|
defer s.mutex.Unlock()
|
||||||
@@ -208,8 +217,7 @@ func (dict *ConcurrentDict) PutIfExistsWithLock(key string, val interface{}) (re
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
|
|
||||||
if _, ok := s.m[key]; ok {
|
if _, ok := s.m[key]; ok {
|
||||||
@@ -224,8 +232,7 @@ func (dict *ConcurrentDict) Remove(key string) (val interface{}, result int) {
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
s.mutex.Lock()
|
s.mutex.Lock()
|
||||||
defer s.mutex.Unlock()
|
defer s.mutex.Unlock()
|
||||||
@@ -242,8 +249,7 @@ func (dict *ConcurrentDict) RemoveWithLock(key string) (val interface{}, result
|
|||||||
if dict == nil {
|
if dict == nil {
|
||||||
panic("dict is nil")
|
panic("dict is nil")
|
||||||
}
|
}
|
||||||
hashCode := fnv32(key)
|
index := dict.spread(key)
|
||||||
index := dict.spread(hashCode)
|
|
||||||
s := dict.getShard(index)
|
s := dict.getShard(index)
|
||||||
|
|
||||||
if val, ok := s.m[key]; ok {
|
if val, ok := s.m[key]; ok {
|
||||||
@@ -381,7 +387,7 @@ func (dict *ConcurrentDict) Clear() {
|
|||||||
func (dict *ConcurrentDict) toLockIndices(keys []string, reverse bool) []uint32 {
|
func (dict *ConcurrentDict) toLockIndices(keys []string, reverse bool) []uint32 {
|
||||||
indexMap := make(map[uint32]struct{})
|
indexMap := make(map[uint32]struct{})
|
||||||
for _, key := range keys {
|
for _, key := range keys {
|
||||||
index := dict.spread(fnv32(key))
|
index := dict.spread(key)
|
||||||
indexMap[index] = struct{}{}
|
indexMap[index] = struct{}{}
|
||||||
}
|
}
|
||||||
indices := make([]uint32, 0, len(indexMap))
|
indices := make([]uint32, 0, len(indexMap))
|
||||||
@@ -403,7 +409,7 @@ func (dict *ConcurrentDict) RWLocks(writeKeys []string, readKeys []string) {
|
|||||||
indices := dict.toLockIndices(keys, false)
|
indices := dict.toLockIndices(keys, false)
|
||||||
writeIndexSet := make(map[uint32]struct{})
|
writeIndexSet := make(map[uint32]struct{})
|
||||||
for _, wKey := range writeKeys {
|
for _, wKey := range writeKeys {
|
||||||
idx := dict.spread(fnv32(wKey))
|
idx := dict.spread(wKey)
|
||||||
writeIndexSet[idx] = struct{}{}
|
writeIndexSet[idx] = struct{}{}
|
||||||
}
|
}
|
||||||
for _, index := range indices {
|
for _, index := range indices {
|
||||||
@@ -423,7 +429,7 @@ func (dict *ConcurrentDict) RWUnLocks(writeKeys []string, readKeys []string) {
|
|||||||
indices := dict.toLockIndices(keys, true)
|
indices := dict.toLockIndices(keys, true)
|
||||||
writeIndexSet := make(map[uint32]struct{})
|
writeIndexSet := make(map[uint32]struct{})
|
||||||
for _, wKey := range writeKeys {
|
for _, wKey := range writeKeys {
|
||||||
idx := dict.spread(fnv32(wKey))
|
idx := dict.spread(wKey)
|
||||||
writeIndexSet[idx] = struct{}{}
|
writeIndexSet[idx] = struct{}{}
|
||||||
}
|
}
|
||||||
for _, index := range indices {
|
for _, index := range indices {
|
||||||
|
@@ -21,6 +21,17 @@ func Make(members ...string) *Set {
|
|||||||
return set
|
return set
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MakeConcurrentSafe creates a new set which is concurrent safe
|
||||||
|
func MakeConcurrentSafe(members ...string) *Set {
|
||||||
|
set := &Set{
|
||||||
|
dict: dict.MakeConcurrent(1),
|
||||||
|
}
|
||||||
|
for _, member := range members {
|
||||||
|
set.Add(member)
|
||||||
|
}
|
||||||
|
return set
|
||||||
|
}
|
||||||
|
|
||||||
// Add adds member into set
|
// Add adds member into set
|
||||||
func (set *Set) Add(val string) int {
|
func (set *Set) Add(val string) int {
|
||||||
return set.dict.Put(val, nil)
|
return set.dict.Put(val, nil)
|
||||||
|
19
go.mod
19
go.mod
@@ -1,7 +1,24 @@
|
|||||||
module github.com/hdt3213/godis
|
module github.com/hdt3213/godis
|
||||||
|
|
||||||
go 1.17
|
go 1.18
|
||||||
|
|
||||||
require (
|
require (
|
||||||
|
github.com/hashicorp/raft v1.7.0
|
||||||
|
github.com/hashicorp/raft-boltdb v0.0.0-20231211162105-6c830fa4535e
|
||||||
github.com/hdt3213/rdb v1.0.10
|
github.com/hdt3213/rdb v1.0.10
|
||||||
|
golang.org/x/tools v0.14.0
|
||||||
|
)
|
||||||
|
|
||||||
|
require (
|
||||||
|
github.com/armon/go-metrics v0.4.1 // indirect
|
||||||
|
github.com/boltdb/bolt v1.3.1 // indirect
|
||||||
|
github.com/fatih/color v1.13.0 // indirect
|
||||||
|
github.com/hashicorp/go-hclog v1.6.2 // indirect
|
||||||
|
github.com/hashicorp/go-immutable-radix v1.0.0 // indirect
|
||||||
|
github.com/hashicorp/go-msgpack v0.5.5 // indirect
|
||||||
|
github.com/hashicorp/go-msgpack/v2 v2.1.1 // indirect
|
||||||
|
github.com/hashicorp/golang-lru v0.5.0 // indirect
|
||||||
|
github.com/mattn/go-colorable v0.1.12 // indirect
|
||||||
|
github.com/mattn/go-isatty v0.0.14 // indirect
|
||||||
|
golang.org/x/sys v0.13.0 // indirect
|
||||||
)
|
)
|
||||||
|
135
go.sum
135
go.sum
@@ -1,26 +1,157 @@
|
|||||||
|
github.com/DataDog/datadog-go v2.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ=
|
||||||
|
github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ=
|
||||||
|
github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
|
||||||
|
github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
|
||||||
|
github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
|
||||||
|
github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
|
||||||
|
github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878/go.mod h1:3AMJUQhVx52RsWOnlkpikZr01T/yAVN2gn0861vByNg=
|
||||||
|
github.com/armon/go-metrics v0.3.8/go.mod h1:4O98XIr/9W0sxpJ8UaYkvjk10Iff7SnFrb4QAOwNTFc=
|
||||||
|
github.com/armon/go-metrics v0.4.1 h1:hR91U9KYmb6bLBYLQjyM+3j+rcd/UhE+G78SFnF8gJA=
|
||||||
|
github.com/armon/go-metrics v0.4.1/go.mod h1:E6amYzXo6aW1tqzoZGT755KkbgrJsSdpwZ+3JqfkOG4=
|
||||||
|
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
|
||||||
|
github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
|
||||||
|
github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
|
||||||
|
github.com/boltdb/bolt v1.3.1 h1:JQmyP4ZBrce+ZQu0dY660FMfatumYDLun9hBCUVIkF4=
|
||||||
|
github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps=
|
||||||
github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM=
|
github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM=
|
||||||
github.com/bytedance/sonic v1.8.7/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U=
|
github.com/bytedance/sonic v1.8.7/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U=
|
||||||
|
github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
|
||||||
github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY=
|
github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY=
|
||||||
github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk=
|
github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk=
|
||||||
|
github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag=
|
||||||
|
github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I=
|
||||||
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
||||||
|
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
|
||||||
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
||||||
github.com/emirpasic/gods v1.18.1/go.mod h1:8tpGGwCnJ5H4r6BWwaV6OrWmMoPhUl5jm/FMNAnJvWQ=
|
github.com/emirpasic/gods v1.18.1/go.mod h1:8tpGGwCnJ5H4r6BWwaV6OrWmMoPhUl5jm/FMNAnJvWQ=
|
||||||
|
github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w=
|
||||||
|
github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk=
|
||||||
|
github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
|
||||||
|
github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
|
||||||
|
github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
|
||||||
|
github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
|
||||||
|
github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
|
||||||
|
github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
|
||||||
|
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||||
|
github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||||
|
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||||
|
github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
|
||||||
|
github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
|
||||||
|
github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
|
||||||
|
github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80=
|
||||||
|
github.com/hashicorp/go-hclog v0.9.1/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrjA0H7acj2lQ=
|
||||||
|
github.com/hashicorp/go-hclog v1.6.2 h1:NOtoftovWkDheyUM/8JW3QMiXyxJK3uHRK7wV04nD2I=
|
||||||
|
github.com/hashicorp/go-hclog v1.6.2/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M=
|
||||||
|
github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0=
|
||||||
|
github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
|
||||||
|
github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI=
|
||||||
|
github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM=
|
||||||
|
github.com/hashicorp/go-msgpack/v2 v2.1.1 h1:xQEY9yB2wnHitoSzk/B9UjXWRQ67QKu5AOm8aFp8N3I=
|
||||||
|
github.com/hashicorp/go-msgpack/v2 v2.1.1/go.mod h1:upybraOAblm4S7rx0+jeNy+CWWhzywQsSRV5033mMu4=
|
||||||
|
github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs=
|
||||||
|
github.com/hashicorp/go-uuid v1.0.0 h1:RS8zrF7PhGwyNPOtxSClXXj9HA8feRnJzgnI1RJCSnM=
|
||||||
|
github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
|
||||||
|
github.com/hashicorp/golang-lru v0.5.0 h1:CL2msUPvZTLb5O648aiLNJw3hnBxN2+1Jq8rCOH9wdo=
|
||||||
|
github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
|
||||||
|
github.com/hashicorp/raft v1.1.0/go.mod h1:4Ak7FSPnuvmb0GV6vgIAJ4vYT4bek9bb6Q+7HVbyzqM=
|
||||||
|
github.com/hashicorp/raft v1.7.0 h1:4u24Qn6lQ6uwziM++UgsyiT64Q8GyRn43CV41qPiz1o=
|
||||||
|
github.com/hashicorp/raft v1.7.0/go.mod h1:N1sKh6Vn47mrWvEArQgILTyng8GoDRNYlgKyK7PMjs0=
|
||||||
|
github.com/hashicorp/raft-boltdb v0.0.0-20231211162105-6c830fa4535e h1:SK4y8oR4ZMHPvwVHryKI88kJPJda4UyWYvG5A6iEQxc=
|
||||||
|
github.com/hashicorp/raft-boltdb v0.0.0-20231211162105-6c830fa4535e/go.mod h1:EMz/UIuG93P0MBeHh6CbXQAEe8ckVJLZjhD17lBzK5Q=
|
||||||
github.com/hdt3213/rdb v1.0.10 h1:j0wJv6Cp1faMH3v5+u5SYa0MfBGOnOc5nn+JEYbIVxA=
|
github.com/hdt3213/rdb v1.0.10 h1:j0wJv6Cp1faMH3v5+u5SYa0MfBGOnOc5nn+JEYbIVxA=
|
||||||
github.com/hdt3213/rdb v1.0.10/go.mod h1:A1RWBSb4QGdX8fNs2bSoWxkzcWlWGbCC7OgOTFhPG+k=
|
github.com/hdt3213/rdb v1.0.10/go.mod h1:A1RWBSb4QGdX8fNs2bSoWxkzcWlWGbCC7OgOTFhPG+k=
|
||||||
|
github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
|
||||||
|
github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
|
||||||
|
github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w=
|
||||||
github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg=
|
github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg=
|
||||||
|
github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
|
||||||
|
github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
|
||||||
|
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
|
||||||
|
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
|
||||||
|
github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
|
||||||
|
github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc=
|
||||||
|
github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40=
|
||||||
|
github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4=
|
||||||
|
github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU=
|
||||||
|
github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y=
|
||||||
|
github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94=
|
||||||
|
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
|
||||||
|
github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
|
||||||
|
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
|
||||||
|
github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
|
||||||
|
github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
|
||||||
|
github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
|
||||||
|
github.com/pascaldekloe/goe v0.1.0 h1:cBOtyMzM9HTpWjXfbbunk26uA6nG3a8n06Wieeh0MwY=
|
||||||
|
github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
|
||||||
|
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
|
||||||
|
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
|
||||||
|
github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
|
||||||
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
|
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
|
||||||
github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8=
|
github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
|
||||||
github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o=
|
github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM=
|
||||||
|
github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo=
|
||||||
|
github.com/prometheus/client_golang v1.4.0/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU=
|
||||||
|
github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
|
||||||
|
github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
|
||||||
|
github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
|
||||||
|
github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro=
|
||||||
|
github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
|
||||||
|
github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4=
|
||||||
|
github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
|
||||||
|
github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
|
||||||
|
github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
|
||||||
|
github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A=
|
||||||
|
github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
|
||||||
|
github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE=
|
||||||
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
|
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
|
||||||
|
github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
|
||||||
github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
|
github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
|
||||||
github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=
|
github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=
|
||||||
|
github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
|
||||||
|
github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
|
||||||
|
github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
|
||||||
github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
|
github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
|
||||||
github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
|
github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
|
||||||
|
github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals=
|
||||||
github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
|
github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
|
||||||
github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4=
|
github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4=
|
||||||
|
github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk=
|
||||||
|
github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM=
|
||||||
github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08=
|
github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08=
|
||||||
golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8=
|
golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8=
|
||||||
|
golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
|
||||||
|
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
|
||||||
|
golang.org/x/mod v0.13.0 h1:I/DsJXRlw/8l/0c24sM9yb0T4z9liZTduXvdAWYiysY=
|
||||||
|
golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
|
golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||||
|
golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||||
|
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
|
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
|
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||||
|
golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
|
golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
|
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||||
|
golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||||
|
golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||||
|
golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||||
|
golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||||
|
golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||||
|
golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||||
|
golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||||
|
golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE=
|
||||||
|
golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||||
|
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||||
|
golang.org/x/tools v0.14.0 h1:jvNa2pY0M4r62jkRQ6RwEZZyPcymeL9XZMLBbV7U2nc=
|
||||||
|
golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg=
|
||||||
|
golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
|
||||||
|
gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
|
||||||
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||||
|
gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||||
|
gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||||
|
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||||
|
gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||||
|
gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||||
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
|
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
|
||||||
|
gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
|
||||||
gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
|
gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
|
||||||
rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4=
|
rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4=
|
||||||
|
@@ -63,6 +63,10 @@ func MakeClient(addr string) (*Client, error) {
|
|||||||
}, nil
|
}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (client *Client) RemoteAddress() string {
|
||||||
|
return client.addr
|
||||||
|
}
|
||||||
|
|
||||||
// Start starts asynchronous goroutines
|
// Start starts asynchronous goroutines
|
||||||
func (client *Client) Start() {
|
func (client *Client) Start() {
|
||||||
client.ticker = time.NewTicker(10 * time.Second)
|
client.ticker = time.NewTicker(10 * time.Second)
|
||||||
|
@@ -2,8 +2,10 @@ package protocol
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
"github.com/hdt3213/godis/interface/redis"
|
"errors"
|
||||||
"strconv"
|
"strconv"
|
||||||
|
|
||||||
|
"github.com/hdt3213/godis/interface/redis"
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
@@ -176,6 +178,17 @@ func IsErrorReply(reply redis.Reply) bool {
|
|||||||
return reply.ToBytes()[0] == '-'
|
return reply.ToBytes()[0] == '-'
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func Try2ErrorReply(reply redis.Reply) error {
|
||||||
|
str := string(reply.ToBytes())
|
||||||
|
if len(str) == 0 {
|
||||||
|
return errors.New("empty reply")
|
||||||
|
}
|
||||||
|
if str[0] != '-' {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
return errors.New(str[1:])
|
||||||
|
}
|
||||||
|
|
||||||
// ToBytes marshal redis.Reply
|
// ToBytes marshal redis.Reply
|
||||||
func (r *StandardErrReply) ToBytes() []byte {
|
func (r *StandardErrReply) ToBytes() []byte {
|
||||||
return []byte("-" + r.Status + CRLF)
|
return []byte("-" + r.Status + CRLF)
|
||||||
|
Reference in New Issue
Block a user