mirror of
https://github.com/HDT3213/godis.git
synced 2025-11-02 04:52:49 +08:00
Merge commit '0bf068f51cb79170dddf8248f21cb4b540107711'
This commit is contained in:
@@ -2,234 +2,38 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"runtime/debug"
|
||||
"strings"
|
||||
|
||||
"github.com/hdt3213/rdb/core"
|
||||
|
||||
"github.com/hdt3213/godis/config"
|
||||
database2 "github.com/hdt3213/godis/database"
|
||||
"github.com/hdt3213/godis/datastruct/dict"
|
||||
"github.com/hdt3213/godis/datastruct/set"
|
||||
"github.com/hdt3213/godis/interface/database"
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/idgenerator"
|
||||
"github.com/hdt3213/godis/lib/logger"
|
||||
"github.com/hdt3213/godis/redis/parser"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"os"
|
||||
"path"
|
||||
"sync"
|
||||
|
||||
_ "github.com/hdt3213/godis/cluster/commands" // register nodes
|
||||
"github.com/hdt3213/godis/cluster/core"
|
||||
"github.com/hdt3213/godis/cluster/raft"
|
||||
"github.com/hdt3213/godis/config"
|
||||
"github.com/hdt3213/godis/lib/logger"
|
||||
)
|
||||
|
||||
// Cluster represents a node of godis cluster
|
||||
// it holds part of data and coordinates other nodes to finish transactions
|
||||
type Cluster struct {
|
||||
self string
|
||||
addr string
|
||||
db database.DBEngine
|
||||
transactions *dict.SimpleDict // id -> Transaction
|
||||
transactionMu sync.RWMutex
|
||||
topology topology
|
||||
slotMu sync.RWMutex
|
||||
slots map[uint32]*hostSlot
|
||||
idGenerator *idgenerator.IDGenerator
|
||||
|
||||
clientFactory clientFactory
|
||||
}
|
||||
|
||||
type peerClient interface {
|
||||
Send(args [][]byte) redis.Reply
|
||||
}
|
||||
|
||||
type peerStream interface {
|
||||
Stream() <-chan *parser.Payload
|
||||
Close() error
|
||||
}
|
||||
|
||||
type clientFactory interface {
|
||||
GetPeerClient(peerAddr string) (peerClient, error)
|
||||
ReturnPeerClient(peerAddr string, peerClient peerClient) error
|
||||
NewStream(peerAddr string, cmdLine CmdLine) (peerStream, error)
|
||||
Close() error
|
||||
}
|
||||
|
||||
const (
|
||||
slotStateHost = iota
|
||||
slotStateImporting
|
||||
slotStateMovingOut
|
||||
)
|
||||
|
||||
// hostSlot stores status of host which hosted by current node
|
||||
type hostSlot struct {
|
||||
state uint32
|
||||
mu sync.RWMutex
|
||||
// OldNodeID is the node which is moving out this slot
|
||||
// only valid during slot is importing
|
||||
oldNodeID string
|
||||
// OldNodeID is the node which is importing this slot
|
||||
// only valid during slot is moving out
|
||||
newNodeID string
|
||||
|
||||
/* importedKeys stores imported keys during migrating progress
|
||||
* While this slot is migrating, if importedKeys does not have the given key, then current node will import key before execute commands
|
||||
*
|
||||
* In a migrating slot, the slot on the old node is immutable, we only delete a key in the new node.
|
||||
* Therefore, we must distinguish between non-migrated key and deleted key.
|
||||
* Even if a key has been deleted, it still exists in importedKeys, so we can distinguish between non-migrated and deleted.
|
||||
*/
|
||||
importedKeys *set.Set
|
||||
// keys stores all keys in this slot
|
||||
// Cluster.makeInsertCallback and Cluster.makeDeleteCallback will keep keys up to time
|
||||
keys *set.Set
|
||||
}
|
||||
|
||||
// if only one node involved in a transaction, just execute the command don't apply tcc procedure
|
||||
var allowFastTransaction = true
|
||||
type Cluster = core.Cluster
|
||||
|
||||
// MakeCluster creates and starts a node of cluster
|
||||
func MakeCluster() *Cluster {
|
||||
cluster := &Cluster{
|
||||
self: config.Properties.Self,
|
||||
addr: config.Properties.AnnounceAddress(),
|
||||
db: database2.NewStandaloneServer(),
|
||||
transactions: dict.MakeSimple(),
|
||||
idGenerator: idgenerator.MakeGenerator(config.Properties.Self),
|
||||
clientFactory: newDefaultClientFactory(),
|
||||
}
|
||||
topologyPersistFile := path.Join(config.Properties.Dir, config.Properties.ClusterConfigFile)
|
||||
cluster.topology = newRaft(cluster, topologyPersistFile)
|
||||
cluster.db.SetKeyInsertedCallback(cluster.makeInsertCallback())
|
||||
cluster.db.SetKeyDeletedCallback(cluster.makeDeleteCallback())
|
||||
cluster.slots = make(map[uint32]*hostSlot)
|
||||
var err error
|
||||
if topologyPersistFile != "" && fileExists(topologyPersistFile) {
|
||||
err = cluster.LoadConfig()
|
||||
} else if config.Properties.ClusterAsSeed {
|
||||
err = cluster.startAsSeed(config.Properties.AnnounceAddress())
|
||||
} else {
|
||||
err = cluster.Join(config.Properties.ClusterSeed)
|
||||
}
|
||||
raftPath := path.Join(config.Properties.Dir, "raft")
|
||||
err := os.MkdirAll(raftPath, os.ModePerm)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
cluster, err := core.NewCluster(&core.Config{
|
||||
RaftConfig: raft.RaftConfig{
|
||||
RedisAdvertiseAddr: config.Properties.AnnounceAddress(),
|
||||
RaftListenAddr: config.Properties.RaftListenAddr,
|
||||
RaftAdvertiseAddr: config.Properties.RaftAdvertiseAddr,
|
||||
Dir: raftPath,
|
||||
},
|
||||
StartAsSeed: config.Properties.ClusterAsSeed,
|
||||
JoinAddress: config.Properties.ClusterSeed,
|
||||
})
|
||||
if err != nil {
|
||||
logger.Error(err.Error())
|
||||
panic(err)
|
||||
}
|
||||
return cluster
|
||||
}
|
||||
|
||||
// CmdFunc represents the handler of a redis command
|
||||
type CmdFunc func(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply
|
||||
|
||||
// Close stops current node of cluster
|
||||
func (cluster *Cluster) Close() {
|
||||
_ = cluster.topology.Close()
|
||||
cluster.db.Close()
|
||||
cluster.clientFactory.Close()
|
||||
}
|
||||
|
||||
func isAuthenticated(c redis.Connection) bool {
|
||||
if config.Properties.RequirePass == "" {
|
||||
return true
|
||||
}
|
||||
return c.GetPassword() == config.Properties.RequirePass
|
||||
}
|
||||
|
||||
// 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 == "info" {
|
||||
if ser, ok := cluster.db.(*database2.Server); ok {
|
||||
return database2.Info(ser, cmdLine[1:])
|
||||
}
|
||||
}
|
||||
if cmdName == "auth" {
|
||||
return database2.Auth(c, cmdLine[1:])
|
||||
}
|
||||
if !isAuthenticated(c) {
|
||||
return protocol.MakeErrReply("NOAUTH Authentication required")
|
||||
}
|
||||
|
||||
if cmdName == "dbsize" {
|
||||
if ser, ok := cluster.db.(*database2.Server); ok {
|
||||
return database2.DbSize(c, ser)
|
||||
}
|
||||
}
|
||||
|
||||
if cmdName == "multi" {
|
||||
if len(cmdLine) != 1 {
|
||||
return protocol.MakeArgNumErrReply(cmdName)
|
||||
}
|
||||
return database2.StartMulti(c)
|
||||
} else if cmdName == "discard" {
|
||||
if len(cmdLine) != 1 {
|
||||
return protocol.MakeArgNumErrReply(cmdName)
|
||||
}
|
||||
return database2.DiscardMulti(c)
|
||||
} else if cmdName == "exec" {
|
||||
if len(cmdLine) != 1 {
|
||||
return protocol.MakeArgNumErrReply(cmdName)
|
||||
}
|
||||
return execMulti(cluster, c, nil)
|
||||
} else if cmdName == "select" {
|
||||
return protocol.MakeErrReply("select not supported in cluster")
|
||||
}
|
||||
if c != nil && c.InMultiState() {
|
||||
return database2.EnqueueCmd(c, cmdLine)
|
||||
}
|
||||
cmdFunc, ok := router[cmdName]
|
||||
if !ok {
|
||||
return protocol.MakeErrReply("ERR unknown command '" + cmdName + "', or not supported in cluster mode")
|
||||
}
|
||||
result = cmdFunc(cluster, c, cmdLine)
|
||||
return
|
||||
}
|
||||
|
||||
// AfterClientClose does some clean after client close connection
|
||||
func (cluster *Cluster) AfterClientClose(c redis.Connection) {
|
||||
cluster.db.AfterClientClose(c)
|
||||
}
|
||||
|
||||
func (cluster *Cluster) LoadRDB(dec *core.Decoder) error {
|
||||
return cluster.db.LoadRDB(dec)
|
||||
}
|
||||
|
||||
func (cluster *Cluster) makeInsertCallback() database.KeyEventCallback {
|
||||
return func(dbIndex int, key string, entity *database.DataEntity) {
|
||||
slotId := getSlot(key)
|
||||
cluster.slotMu.RLock()
|
||||
slot, ok := cluster.slots[slotId]
|
||||
cluster.slotMu.RUnlock()
|
||||
// As long as the command is executed, we should update slot.keys regardless of slot.state
|
||||
if ok {
|
||||
slot.mu.Lock()
|
||||
defer slot.mu.Unlock()
|
||||
slot.keys.Add(key)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (cluster *Cluster) makeDeleteCallback() database.KeyEventCallback {
|
||||
return func(dbIndex int, key string, entity *database.DataEntity) {
|
||||
slotId := getSlot(key)
|
||||
cluster.slotMu.RLock()
|
||||
slot, ok := cluster.slots[slotId]
|
||||
cluster.slotMu.RUnlock()
|
||||
// As long as the command is executed, we should update slot.keys regardless of slot.state
|
||||
if ok {
|
||||
slot.mu.Lock()
|
||||
defer slot.mu.Unlock()
|
||||
slot.keys.Remove(key)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func fileExists(filename string) bool {
|
||||
info, err := os.Stat(filename)
|
||||
return err == nil && !info.IsDir()
|
||||
}
|
||||
|
||||
@@ -1,86 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
)
|
||||
|
||||
// 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.self {
|
||||
// to self db
|
||||
return cluster.Exec(c, cmdLine)
|
||||
}
|
||||
// peerId is peer.Addr
|
||||
cli, err := cluster.clientFactory.GetPeerClient(peerId)
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply(err.Error())
|
||||
}
|
||||
defer func() {
|
||||
_ = cluster.clientFactory.ReturnPeerClient(peerId, cli)
|
||||
}()
|
||||
return cli.Send(cmdLine)
|
||||
}
|
||||
|
||||
// relayByKey function relays command to peer
|
||||
// use routeKey to determine peer node
|
||||
func (cluster *Cluster) relayByKey(routeKey string, c redis.Connection, args [][]byte) redis.Reply {
|
||||
slotId := getSlot(routeKey)
|
||||
peer := cluster.pickNode(slotId)
|
||||
return cluster.relay(peer.ID, c, args)
|
||||
}
|
||||
|
||||
// broadcast function broadcasts command to all node in cluster
|
||||
func (cluster *Cluster) broadcast(c redis.Connection, args [][]byte) map[string]redis.Reply {
|
||||
result := make(map[string]redis.Reply)
|
||||
for _, node := range cluster.topology.GetNodes() {
|
||||
reply := cluster.relay(node.ID, c, args)
|
||||
result[node.Addr] = reply
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
// ensureKey will migrate key to current node if the key is in a slot migrating to current node
|
||||
// invoker should provide with locks of key
|
||||
func (cluster *Cluster) ensureKey(key string) protocol.ErrorReply {
|
||||
slotId := getSlot(key)
|
||||
cluster.slotMu.RLock()
|
||||
slot := cluster.slots[slotId]
|
||||
cluster.slotMu.RUnlock()
|
||||
if slot == nil {
|
||||
return nil
|
||||
}
|
||||
if slot.state != slotStateImporting || slot.importedKeys.Has(key) {
|
||||
return nil
|
||||
}
|
||||
resp := cluster.relay(slot.oldNodeID, connection.NewFakeConn(), utils.ToCmdLine("DumpKey_", key))
|
||||
if protocol.IsErrorReply(resp) {
|
||||
return resp.(protocol.ErrorReply)
|
||||
}
|
||||
if protocol.IsEmptyMultiBulkReply(resp) {
|
||||
slot.importedKeys.Add(key)
|
||||
return nil
|
||||
}
|
||||
dumpResp := resp.(*protocol.MultiBulkReply)
|
||||
if len(dumpResp.Args) != 2 {
|
||||
return protocol.MakeErrReply("illegal dump key response")
|
||||
}
|
||||
// reuse copy to command ^_^
|
||||
resp = cluster.db.Exec(connection.NewFakeConn(), [][]byte{
|
||||
[]byte("CopyTo"), []byte(key), dumpResp.Args[0], dumpResp.Args[1],
|
||||
})
|
||||
if protocol.IsErrorReply(resp) {
|
||||
return resp.(protocol.ErrorReply)
|
||||
}
|
||||
slot.importedKeys.Add(key)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (cluster *Cluster) ensureKeyWithoutLock(key string) protocol.ErrorReply {
|
||||
cluster.db.RWLocks(0, []string{key}, nil)
|
||||
defer cluster.db.RWUnLocks(0, []string{key}, nil)
|
||||
return cluster.ensureKey(key)
|
||||
}
|
||||
@@ -1,58 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/config"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestExec(t *testing.T) {
|
||||
testCluster := testCluster[0]
|
||||
conn := connection.NewFakeConn()
|
||||
for i := 0; i < 1000; i++ {
|
||||
key := RandString(4)
|
||||
value := RandString(4)
|
||||
testCluster.Exec(conn, toArgs("SET", key, value))
|
||||
ret := testCluster.Exec(conn, toArgs("GET", key))
|
||||
asserts.AssertBulkReply(t, ret, value)
|
||||
}
|
||||
}
|
||||
|
||||
func TestAuth(t *testing.T) {
|
||||
passwd := utils.RandString(10)
|
||||
config.Properties.RequirePass = passwd
|
||||
defer func() {
|
||||
config.Properties.RequirePass = ""
|
||||
}()
|
||||
conn := connection.NewFakeConn()
|
||||
testCluster := testCluster[0]
|
||||
ret := testCluster.Exec(conn, toArgs("GET", "a"))
|
||||
asserts.AssertErrReply(t, ret, "NOAUTH Authentication required")
|
||||
ret = testCluster.Exec(conn, toArgs("AUTH", passwd))
|
||||
asserts.AssertStatusReply(t, ret, "OK")
|
||||
ret = testCluster.Exec(conn, toArgs("GET", "a"))
|
||||
asserts.AssertNotError(t, ret)
|
||||
}
|
||||
|
||||
func TestRelay(t *testing.T) {
|
||||
testNodeA := testCluster[1]
|
||||
key := RandString(4)
|
||||
value := RandString(4)
|
||||
conn := connection.NewFakeConn()
|
||||
ret := testNodeA.relay(addresses[1], conn, toArgs("SET", key, value))
|
||||
asserts.AssertNotError(t, ret)
|
||||
ret = testNodeA.relay(addresses[1], conn, toArgs("GET", key))
|
||||
asserts.AssertBulkReply(t, ret, value)
|
||||
}
|
||||
|
||||
func TestBroadcast(t *testing.T) {
|
||||
testCluster2 := testCluster[0]
|
||||
key := RandString(4)
|
||||
value := RandString(4)
|
||||
rets := testCluster2.broadcast(connection.NewFakeConn(), toArgs("SET", key, value))
|
||||
for _, v := range rets {
|
||||
asserts.AssertNotError(t, v)
|
||||
}
|
||||
}
|
||||
95
cluster/commands/default.go
Normal file
95
cluster/commands/default.go
Normal file
@@ -0,0 +1,95 @@
|
||||
package commands
|
||||
|
||||
import "github.com/hdt3213/godis/cluster/core"
|
||||
|
||||
func RegisterCommands() {
|
||||
defaultCmds := []string{
|
||||
"expire",
|
||||
"expireAt",
|
||||
"pExpire",
|
||||
"pExpireAt",
|
||||
"ttl",
|
||||
"PTtl",
|
||||
"persist",
|
||||
"exists",
|
||||
"type",
|
||||
"set",
|
||||
"setNx",
|
||||
"setEx",
|
||||
"pSetEx",
|
||||
"get",
|
||||
"getEx",
|
||||
"getSet",
|
||||
"getDel",
|
||||
"incr",
|
||||
"incrBy",
|
||||
"incrByFloat",
|
||||
"decr",
|
||||
"decrBy",
|
||||
"lPush",
|
||||
"lPushX",
|
||||
"rPush",
|
||||
"rPushX",
|
||||
"LPop",
|
||||
"RPop",
|
||||
"LRem",
|
||||
"LLen",
|
||||
"LIndex",
|
||||
"LSet",
|
||||
"LRange",
|
||||
"HSet",
|
||||
"HSetNx",
|
||||
"HGet",
|
||||
"HExists",
|
||||
"HDel",
|
||||
"HLen",
|
||||
"HStrLen",
|
||||
"HMGet",
|
||||
"HMSet",
|
||||
"HKeys",
|
||||
"HVals",
|
||||
"HGetAll",
|
||||
"HIncrBy",
|
||||
"HIncrByFloat",
|
||||
"HRandField",
|
||||
"SAdd",
|
||||
"SIsMember",
|
||||
"SRem",
|
||||
"SPop",
|
||||
"SCard",
|
||||
"SMembers",
|
||||
"SInter",
|
||||
"SInterStore",
|
||||
"SUnion",
|
||||
"SUnionStore",
|
||||
"SDiff",
|
||||
"SDiffStore",
|
||||
"SRandMember",
|
||||
"ZAdd",
|
||||
"ZScore",
|
||||
"ZIncrBy",
|
||||
"ZRank",
|
||||
"ZCount",
|
||||
"ZRevRank",
|
||||
"ZCard",
|
||||
"ZRange",
|
||||
"ZRevRange",
|
||||
"ZRangeByScore",
|
||||
"ZRevRangeByScore",
|
||||
"ZRem",
|
||||
"ZRemRangeByScore",
|
||||
"ZRemRangeByRank",
|
||||
"GeoAdd",
|
||||
"GeoPos",
|
||||
"GeoDist",
|
||||
"GeoHash",
|
||||
"GeoRadius",
|
||||
"GeoRadiusByMember",
|
||||
"GetVer",
|
||||
"DumpKey",
|
||||
}
|
||||
for _, name := range defaultCmds {
|
||||
core.RegisterDefaultCmd(name)
|
||||
}
|
||||
|
||||
}
|
||||
119
cluster/copy.go
119
cluster/copy.go
@@ -1,119 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"strconv"
|
||||
"strings"
|
||||
)
|
||||
|
||||
const copyToAnotherDBErr = "ERR Copying to another database is not allowed in cluster mode"
|
||||
const noReplace = "NoReplace"
|
||||
const useReplace = "UseReplace"
|
||||
|
||||
// Copy copies the value stored at the source key to the destination key.
|
||||
// the origin and the destination must within the same node.
|
||||
func Copy(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
if len(args) < 3 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'copy' command")
|
||||
}
|
||||
srcKey := string(args[1])
|
||||
destKey := string(args[2])
|
||||
srcNode := cluster.pickNodeAddrByKey(srcKey)
|
||||
destNode := cluster.pickNodeAddrByKey(destKey)
|
||||
replaceFlag := noReplace
|
||||
if len(args) > 3 {
|
||||
for i := 3; i < len(args); i++ {
|
||||
arg := strings.ToLower(string(args[i]))
|
||||
if arg == "db" {
|
||||
return protocol.MakeErrReply(copyToAnotherDBErr)
|
||||
} else if arg == "replace" {
|
||||
replaceFlag = useReplace
|
||||
} else {
|
||||
return protocol.MakeSyntaxErrReply()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if srcNode == destNode {
|
||||
args[0] = []byte("Copy_") // Copy_ will go directly to cluster.DB avoiding infinite recursion
|
||||
return cluster.relay(srcNode, c, args)
|
||||
}
|
||||
groupMap := map[string][]string{
|
||||
srcNode: {srcKey},
|
||||
destNode: {destKey},
|
||||
}
|
||||
|
||||
txID := cluster.idGenerator.NextID()
|
||||
txIDStr := strconv.FormatInt(txID, 10)
|
||||
// prepare Copy from
|
||||
srcPrepareResp := cluster.relay(srcNode, c, makeArgs("Prepare", txIDStr, "CopyFrom", srcKey))
|
||||
if protocol.IsErrorReply(srcPrepareResp) {
|
||||
// rollback src node
|
||||
requestRollback(cluster, c, txID, map[string][]string{srcNode: {srcKey}})
|
||||
return srcPrepareResp
|
||||
}
|
||||
srcPrepareMBR, ok := srcPrepareResp.(*protocol.MultiBulkReply)
|
||||
if !ok || len(srcPrepareMBR.Args) < 2 {
|
||||
requestRollback(cluster, c, txID, map[string][]string{srcNode: {srcKey}})
|
||||
return protocol.MakeErrReply("ERR invalid prepare response")
|
||||
}
|
||||
// prepare Copy to
|
||||
destPrepareResp := cluster.relay(destNode, c, utils.ToCmdLine3("Prepare", []byte(txIDStr),
|
||||
[]byte("CopyTo"), []byte(destKey), srcPrepareMBR.Args[0], srcPrepareMBR.Args[1], []byte(replaceFlag)))
|
||||
if destErr, ok := destPrepareResp.(protocol.ErrorReply); ok {
|
||||
// rollback src node
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
if destErr.Error() == keyExistsErr {
|
||||
return protocol.MakeIntReply(0)
|
||||
}
|
||||
return destPrepareResp
|
||||
}
|
||||
if _, errReply := requestCommit(cluster, c, txID, groupMap); errReply != nil {
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
return errReply
|
||||
}
|
||||
return protocol.MakeIntReply(1)
|
||||
}
|
||||
|
||||
// prepareCopyFrom is prepare-function for CopyFrom, see prepareFuncMap
|
||||
func prepareCopyFrom(cluster *Cluster, conn redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if len(cmdLine) != 2 {
|
||||
return protocol.MakeArgNumErrReply("CopyFrom")
|
||||
}
|
||||
key := string(cmdLine[1])
|
||||
existResp := cluster.db.ExecWithLock(conn, utils.ToCmdLine("Exists", key))
|
||||
if protocol.IsErrorReply(existResp) {
|
||||
return existResp
|
||||
}
|
||||
existIntResp := existResp.(*protocol.IntReply)
|
||||
if existIntResp.Code == 0 {
|
||||
return protocol.MakeErrReply("ERR no such key")
|
||||
}
|
||||
return cluster.db.ExecWithLock(conn, utils.ToCmdLine2("DumpKey", key))
|
||||
}
|
||||
|
||||
func prepareCopyTo(cluster *Cluster, conn redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if len(cmdLine) != 5 {
|
||||
return protocol.MakeArgNumErrReply("CopyTo")
|
||||
}
|
||||
key := string(cmdLine[1])
|
||||
replaceFlag := string(cmdLine[4])
|
||||
existResp := cluster.db.ExecWithLock(conn, utils.ToCmdLine("Exists", key))
|
||||
if protocol.IsErrorReply(existResp) {
|
||||
return existResp
|
||||
}
|
||||
existIntResp := existResp.(*protocol.IntReply)
|
||||
if existIntResp.Code == 1 {
|
||||
if replaceFlag == noReplace {
|
||||
return protocol.MakeErrReply(keyExistsErr)
|
||||
}
|
||||
}
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
|
||||
func init() {
|
||||
registerPrepareFunc("CopyFrom", prepareCopyFrom)
|
||||
registerPrepareFunc("CopyTo", prepareCopyTo)
|
||||
}
|
||||
@@ -1,128 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestCopy(t *testing.T) {
|
||||
conn := new(connection.FakeConn)
|
||||
testNodeA := testCluster[0]
|
||||
testNodeB := testCluster[1]
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("FlushALL"))
|
||||
testNodeB.Exec(conn, utils.ToCmdLine("FlushALL"))
|
||||
|
||||
// cross node copy
|
||||
srcKey := "127.0.0.1:6399Bk2r3Sz0V5" // use fix key to ensure hashing to different node
|
||||
destKey := "127.0.0.1:7379CcdC0QOopF"
|
||||
value := utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", srcKey, value))
|
||||
result := Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("GET", srcKey))
|
||||
asserts.AssertBulkReply(t, result, value)
|
||||
result = testNodeB.Exec(conn, utils.ToCmdLine("GET", destKey))
|
||||
asserts.AssertBulkReply(t, result, value)
|
||||
// key exists
|
||||
result = Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey))
|
||||
asserts.AssertIntReply(t, result, 0)
|
||||
// replace
|
||||
value = utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", srcKey, value))
|
||||
result = Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey, "REPLACE"))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("GET", srcKey))
|
||||
asserts.AssertBulkReply(t, result, value)
|
||||
result = testNodeB.Exec(conn, utils.ToCmdLine("GET", destKey))
|
||||
asserts.AssertBulkReply(t, result, value)
|
||||
// test copy expire time
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", srcKey, value, "EX", "1000"))
|
||||
result = Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey, "REPLACE"))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("TTL", srcKey))
|
||||
asserts.AssertIntReplyGreaterThan(t, result, 0)
|
||||
result = testNodeB.Exec(conn, utils.ToCmdLine("TTL", destKey))
|
||||
asserts.AssertIntReplyGreaterThan(t, result, 0)
|
||||
|
||||
// same node copy
|
||||
srcKey = "{" + testNodeA.self + "}" + utils.RandString(10)
|
||||
destKey = "{" + testNodeA.self + "}" + utils.RandString(9)
|
||||
value = utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", srcKey, value))
|
||||
result = Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("GET", srcKey))
|
||||
asserts.AssertBulkReply(t, result, value)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("GET", destKey))
|
||||
asserts.AssertBulkReply(t, result, value)
|
||||
// key exists
|
||||
result = Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey))
|
||||
asserts.AssertIntReply(t, result, 0)
|
||||
// replace
|
||||
value = utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", srcKey, value))
|
||||
result = Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey, "REPLACE"))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("GET", srcKey))
|
||||
asserts.AssertBulkReply(t, result, value)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("GET", destKey))
|
||||
asserts.AssertBulkReply(t, result, value)
|
||||
// test copy expire time
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", srcKey, value, "EX", "1000"))
|
||||
result = Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey, "REPLACE"))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("TTL", srcKey))
|
||||
asserts.AssertIntReplyGreaterThan(t, result, 0)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("TTL", destKey))
|
||||
asserts.AssertIntReplyGreaterThan(t, result, 0)
|
||||
}
|
||||
|
||||
func TestCopyTimeout(t *testing.T) {
|
||||
conn := new(connection.FakeConn)
|
||||
testNodeA := testCluster[0]
|
||||
testNodeB := testCluster[1]
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("FlushALL"))
|
||||
testNodeB.Exec(conn, utils.ToCmdLine("FlushALL"))
|
||||
|
||||
// test src prepare failed
|
||||
timeoutFlags[0] = true
|
||||
srcKey := "127.0.0.1:6399Bk2r3Sz0V5" // use fix key to ensure hashing to different node
|
||||
destKey := "127.0.0.1:7379CcdC0QOopF"
|
||||
value := utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", srcKey, value, "ex", "1000"))
|
||||
result := Rename(testNodeB, conn, utils.ToCmdLine("RENAME", srcKey, destKey))
|
||||
asserts.AssertErrReply(t, result, "ERR timeout")
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("EXISTS", srcKey))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("TTL", srcKey))
|
||||
asserts.AssertIntReplyGreaterThan(t, result, 0)
|
||||
result = testNodeB.Exec(conn, utils.ToCmdLine("EXISTS", destKey))
|
||||
asserts.AssertIntReply(t, result, 0)
|
||||
timeoutFlags[0] = false
|
||||
|
||||
// test dest prepare failed
|
||||
timeoutFlags[1] = true
|
||||
value = utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", srcKey, value, "ex", "1000"))
|
||||
result = Rename(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey))
|
||||
asserts.AssertErrReply(t, result, "ERR timeout")
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("EXISTS", srcKey))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("TTL", srcKey))
|
||||
asserts.AssertIntReplyGreaterThan(t, result, 0)
|
||||
result = testNodeB.Exec(conn, utils.ToCmdLine("EXISTS", destKey))
|
||||
asserts.AssertIntReply(t, result, 0)
|
||||
timeoutFlags[1] = false
|
||||
// Copying to another database
|
||||
srcKey = testNodeA.self + utils.RandString(10)
|
||||
value = utils.RandString(10)
|
||||
destKey = srcKey + utils.RandString(2)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", srcKey, value))
|
||||
result = Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey, destKey, "db", "1"))
|
||||
asserts.AssertErrReply(t, result, copyToAnotherDBErr)
|
||||
|
||||
result = Copy(testNodeA, conn, utils.ToCmdLine("COPY", srcKey))
|
||||
asserts.AssertErrReply(t, result, "ERR wrong number of arguments for 'copy' command")
|
||||
}
|
||||
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)
|
||||
}
|
||||
@@ -1,8 +1,10 @@
|
||||
package cluster
|
||||
package core
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
|
||||
"github.com/hdt3213/godis/config"
|
||||
"github.com/hdt3213/godis/datastruct/dict"
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
@@ -12,9 +14,27 @@ import (
|
||||
"github.com/hdt3213/godis/redis/client"
|
||||
"github.com/hdt3213/godis/redis/parser"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"net"
|
||||
)
|
||||
|
||||
// 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
|
||||
}
|
||||
@@ -24,25 +44,36 @@ var connectionPoolConfig = pool.Config{
|
||||
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) GetPeerClient(peerAddr string) (peerClient, error) {
|
||||
func (factory *defaultClientFactory) BorrowPeerClient(peerAddr string) (peerClient, error) {
|
||||
var connectionPool *pool.Pool
|
||||
raw, ok := factory.nodeConnections.Get(peerAddr)
|
||||
if !ok {
|
||||
creator := func() (interface{}, 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
|
||||
return factory.NewPeerClient(peerAddr)
|
||||
}
|
||||
finalizer := func(x interface{}) {
|
||||
logger.Debug("destroy client")
|
||||
@@ -68,9 +99,14 @@ func (factory *defaultClientFactory) GetPeerClient(peerAddr string) (peerClient,
|
||||
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(peer string, peerClient peerClient) error {
|
||||
raw, ok := factory.nodeConnections.Get(peer)
|
||||
func (factory *defaultClientFactory) ReturnPeerClient(peerClient peerClient) error {
|
||||
raw, ok := factory.nodeConnections.Get(peerClient.RemoteAddress())
|
||||
if !ok {
|
||||
return errors.New("connection pool not found")
|
||||
}
|
||||
@@ -140,3 +176,4 @@ func (factory *defaultClientFactory) Close() error {
|
||||
})
|
||||
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
|
||||
}
|
||||
146
cluster/core/core.go
Normal file
146
cluster/core/core.go
Normal file
@@ -0,0 +1,146 @@
|
||||
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/interface/redis"
|
||||
"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
|
||||
}
|
||||
|
||||
// AfterClientClose does some clean after client close connection
|
||||
func (cluster *Cluster) AfterClientClose(c redis.Connection) {
|
||||
|
||||
}
|
||||
|
||||
func (cluster *Cluster) Close() {
|
||||
cluster.db.Close()
|
||||
err := cluster.raftNode.Close()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
134
cluster/core/core_test.go
Normal file
134
cluster/core/core_test.go
Normal file
@@ -0,0 +1,134 @@
|
||||
package core
|
||||
|
||||
import (
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hdt3213/godis/cluster/raft"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
)
|
||||
|
||||
func TestClusterBootstrap(t *testing.T) {
|
||||
// start leader
|
||||
leaderDir := "test/0"
|
||||
os.RemoveAll(leaderDir)
|
||||
os.MkdirAll(leaderDir, 0777)
|
||||
defer func() {
|
||||
os.RemoveAll(leaderDir)
|
||||
}()
|
||||
RegisterDefaultCmd("get")
|
||||
RegisterDefaultCmd("set")
|
||||
|
||||
// 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
|
||||
|
||||
// set key-values for test
|
||||
testEntries := make(map[string]string)
|
||||
c := connection.NewFakeConn()
|
||||
for i := 0; i < 1000; i++ {
|
||||
key := utils.RandString(10)
|
||||
value := utils.RandString(10)
|
||||
testEntries[key] = value
|
||||
result := leader.Exec(c, utils.ToCmdLine("set", key, value))
|
||||
if !protocol.IsOKReply(result) {
|
||||
t.Errorf("command [set] failed: %s", string(result.ToBytes()))
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// 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)
|
||||
}
|
||||
}
|
||||
|
||||
// set key-values for test
|
||||
for key, value := range testEntries {
|
||||
c := connection.NewFakeConn()
|
||||
result := leader.Exec(c, utils.ToCmdLine("get", key))
|
||||
result2 := result.(*protocol.BulkReply)
|
||||
if string(result2.Arg) != value {
|
||||
t.Errorf("command [get] failed: %s", string(result.ToBytes()))
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
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))
|
||||
}
|
||||
@@ -1,61 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"strconv"
|
||||
)
|
||||
|
||||
// Del atomically removes given writeKeys from cluster, writeKeys can be distributed on any node
|
||||
// if the given writeKeys are distributed on different node, Del will use try-commit-catch to remove them
|
||||
func Del(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
if len(args) < 2 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'del' command")
|
||||
}
|
||||
keys := make([]string, len(args)-1)
|
||||
for i := 1; i < len(args); i++ {
|
||||
keys[i-1] = string(args[i])
|
||||
}
|
||||
groupMap := cluster.groupBy(keys)
|
||||
if len(groupMap) == 1 && allowFastTransaction { // do fast
|
||||
for peer, group := range groupMap { // only one peerKeys
|
||||
return cluster.relay(peer, c, makeArgs("Del_", group...))
|
||||
}
|
||||
}
|
||||
// prepare
|
||||
var errReply redis.Reply
|
||||
txID := cluster.idGenerator.NextID()
|
||||
txIDStr := strconv.FormatInt(txID, 10)
|
||||
rollback := false
|
||||
for peer, peerKeys := range groupMap {
|
||||
peerArgs := []string{txIDStr, "DEL"}
|
||||
peerArgs = append(peerArgs, peerKeys...)
|
||||
var resp redis.Reply
|
||||
resp = cluster.relay(peer, c, makeArgs("Prepare", peerArgs...))
|
||||
if protocol.IsErrorReply(resp) {
|
||||
errReply = resp
|
||||
rollback = true
|
||||
break
|
||||
}
|
||||
}
|
||||
var respList []redis.Reply
|
||||
if rollback {
|
||||
// rollback
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
} else {
|
||||
// commit
|
||||
respList, errReply = requestCommit(cluster, c, txID, groupMap)
|
||||
if errReply != nil {
|
||||
rollback = true
|
||||
}
|
||||
}
|
||||
if !rollback {
|
||||
var deleted int64 = 0
|
||||
for _, resp := range respList {
|
||||
intResp := resp.(*protocol.IntReply)
|
||||
deleted += intResp.Code
|
||||
}
|
||||
return protocol.MakeIntReply(int64(deleted))
|
||||
}
|
||||
return errReply
|
||||
}
|
||||
@@ -1,18 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestDel(t *testing.T) {
|
||||
conn := connection.NewFakeConn()
|
||||
allowFastTransaction = false
|
||||
testNodeA := testCluster[0]
|
||||
testNodeA.Exec(conn, toArgs("SET", "a", "a"))
|
||||
ret := Del(testNodeA, conn, toArgs("DEL", "a", "b", "c"))
|
||||
asserts.AssertNotError(t, ret)
|
||||
ret = testNodeA.Exec(conn, toArgs("GET", "a"))
|
||||
asserts.AssertNullBulk(t, ret)
|
||||
}
|
||||
@@ -1,58 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// fixedTopology is a fixed cluster topology, used for test
|
||||
type fixedTopology struct {
|
||||
mu sync.RWMutex
|
||||
nodeMap map[string]*Node
|
||||
slots []*Slot
|
||||
selfNodeID string
|
||||
}
|
||||
|
||||
func (fixed *fixedTopology) GetSelfNodeID() string {
|
||||
return fixed.selfNodeID
|
||||
}
|
||||
|
||||
func (fixed *fixedTopology) GetNodes() []*Node {
|
||||
fixed.mu.RLock()
|
||||
defer fixed.mu.RUnlock()
|
||||
result := make([]*Node, 0, len(fixed.nodeMap))
|
||||
for _, v := range fixed.nodeMap {
|
||||
result = append(result, v)
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
func (fixed *fixedTopology) GetNode(nodeID string) *Node {
|
||||
fixed.mu.RLock()
|
||||
defer fixed.mu.RUnlock()
|
||||
return fixed.nodeMap[nodeID]
|
||||
}
|
||||
|
||||
func (fixed *fixedTopology) GetSlots() []*Slot {
|
||||
return fixed.slots
|
||||
}
|
||||
|
||||
func (fixed *fixedTopology) StartAsSeed(addr string) protocol.ErrorReply {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fixed *fixedTopology) LoadConfigFile() protocol.ErrorReply {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fixed *fixedTopology) Join(seed string) protocol.ErrorReply {
|
||||
return protocol.MakeErrReply("fixed topology does not support join")
|
||||
}
|
||||
|
||||
func (fixed *fixedTopology) SetSlot(slotIDs []uint32, newNodeID string) protocol.ErrorReply {
|
||||
return protocol.MakeErrReply("fixed topology does not support set slots")
|
||||
}
|
||||
|
||||
func (fixed *fixedTopology) Close() error {
|
||||
return nil
|
||||
}
|
||||
@@ -1,27 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
)
|
||||
|
||||
// FlushDB removes all data in current database
|
||||
func FlushDB(cluster *Cluster, c redis.Connection, cmdLine [][]byte) redis.Reply {
|
||||
replies := cluster.broadcast(c, modifyCmd(cmdLine, "FlushDB_"))
|
||||
var errReply protocol.ErrorReply
|
||||
for _, v := range replies {
|
||||
if protocol.IsErrorReply(v) {
|
||||
errReply = v.(protocol.ErrorReply)
|
||||
break
|
||||
}
|
||||
}
|
||||
if errReply == nil {
|
||||
return &protocol.OkReply{}
|
||||
}
|
||||
return protocol.MakeErrReply("error occurs: " + errReply.Error())
|
||||
}
|
||||
|
||||
// FlushAll removes all data in cluster
|
||||
func FlushAll(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
return FlushDB(cluster, c, args)
|
||||
}
|
||||
181
cluster/mset.go
181
cluster/mset.go
@@ -1,181 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"strconv"
|
||||
)
|
||||
|
||||
const keyExistsErr = "key exists"
|
||||
|
||||
// MGet atomically get multi key-value from cluster, writeKeys can be distributed on any node
|
||||
func MGet(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if len(cmdLine) < 2 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'mget' command")
|
||||
}
|
||||
keys := make([]string, len(cmdLine)-1)
|
||||
for i := 1; i < len(cmdLine); i++ {
|
||||
keys[i-1] = string(cmdLine[i])
|
||||
}
|
||||
|
||||
resultMap := make(map[string][]byte)
|
||||
groupMap := cluster.groupBy(keys)
|
||||
for peer, groupKeys := range groupMap {
|
||||
resp := cluster.relay(peer, c, makeArgs("MGet_", groupKeys...))
|
||||
if protocol.IsErrorReply(resp) {
|
||||
errReply := resp.(protocol.ErrorReply)
|
||||
return protocol.MakeErrReply(fmt.Sprintf("ERR during get %s occurs: %v", groupKeys[0], errReply.Error()))
|
||||
}
|
||||
arrReply, _ := resp.(*protocol.MultiBulkReply)
|
||||
for i, v := range arrReply.Args {
|
||||
key := groupKeys[i]
|
||||
resultMap[key] = v
|
||||
}
|
||||
}
|
||||
result := make([][]byte, len(keys))
|
||||
for i, k := range keys {
|
||||
result[i] = resultMap[k]
|
||||
}
|
||||
return protocol.MakeMultiBulkReply(result)
|
||||
}
|
||||
|
||||
// MSet atomically sets multi key-value in cluster, writeKeys can be distributed on any node
|
||||
func MSet(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
argCount := len(cmdLine) - 1
|
||||
if argCount%2 != 0 || argCount < 1 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'mset' command")
|
||||
}
|
||||
|
||||
size := argCount / 2
|
||||
keys := make([]string, size)
|
||||
valueMap := make(map[string]string)
|
||||
for i := 0; i < size; i++ {
|
||||
keys[i] = string(cmdLine[2*i+1])
|
||||
valueMap[keys[i]] = string(cmdLine[2*i+2])
|
||||
}
|
||||
|
||||
groupMap := cluster.groupBy(keys)
|
||||
if len(groupMap) == 1 && allowFastTransaction { // do fast
|
||||
for peer := range groupMap {
|
||||
return cluster.relay(peer, c, modifyCmd(cmdLine, "MSet_"))
|
||||
}
|
||||
}
|
||||
|
||||
//prepare
|
||||
var errReply redis.Reply
|
||||
txID := cluster.idGenerator.NextID()
|
||||
txIDStr := strconv.FormatInt(txID, 10)
|
||||
rollback := false
|
||||
for peer, group := range groupMap {
|
||||
peerArgs := []string{txIDStr, "MSET"}
|
||||
for _, k := range group {
|
||||
peerArgs = append(peerArgs, k, valueMap[k])
|
||||
}
|
||||
resp := cluster.relay(peer, c, makeArgs("Prepare", peerArgs...))
|
||||
if protocol.IsErrorReply(resp) {
|
||||
errReply = resp
|
||||
rollback = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if rollback {
|
||||
// rollback
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
} else {
|
||||
_, errReply = requestCommit(cluster, c, txID, groupMap)
|
||||
rollback = errReply != nil
|
||||
}
|
||||
if !rollback {
|
||||
return &protocol.OkReply{}
|
||||
}
|
||||
return errReply
|
||||
|
||||
}
|
||||
|
||||
// MSetNX sets multi key-value in database, only if none of the given writeKeys exist and all given writeKeys are on the same node
|
||||
func MSetNX(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
argCount := len(cmdLine) - 1
|
||||
if argCount%2 != 0 || argCount < 1 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'msetnx' command")
|
||||
}
|
||||
|
||||
size := argCount / 2
|
||||
keys := make([]string, size)
|
||||
valueMap := make(map[string]string)
|
||||
for i := 0; i < size; i++ {
|
||||
keys[i] = string(cmdLine[2*i+1])
|
||||
valueMap[keys[i]] = string(cmdLine[2*i+2])
|
||||
}
|
||||
|
||||
groupMap := cluster.groupBy(keys)
|
||||
if len(groupMap) == 1 && allowFastTransaction { // do fast
|
||||
for peer := range groupMap {
|
||||
return cluster.relay(peer, c, modifyCmd(cmdLine, "MSetNX_"))
|
||||
}
|
||||
}
|
||||
|
||||
// prepare procedure:
|
||||
// 1. Normal tcc preparation (undo log and lock related keys)
|
||||
// 2. Peer checks whether any key already exists, If so it will return keyExistsErr. Then coordinator will request rollback over all participated nodes
|
||||
var errReply redis.Reply
|
||||
txID := cluster.idGenerator.NextID()
|
||||
txIDStr := strconv.FormatInt(txID, 10)
|
||||
rollback := false
|
||||
for node, group := range groupMap {
|
||||
nodeArgs := []string{txIDStr, "MSETNX"}
|
||||
for _, k := range group {
|
||||
nodeArgs = append(nodeArgs, k, valueMap[k])
|
||||
}
|
||||
resp := cluster.relay(node, c, makeArgs("Prepare", nodeArgs...))
|
||||
if protocol.IsErrorReply(resp) {
|
||||
re := resp.(protocol.ErrorReply)
|
||||
if re.Error() == keyExistsErr {
|
||||
errReply = protocol.MakeIntReply(0)
|
||||
} else {
|
||||
errReply = resp
|
||||
}
|
||||
rollback = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if rollback {
|
||||
// rollback
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
return errReply
|
||||
}
|
||||
_, errReply = requestCommit(cluster, c, txID, groupMap)
|
||||
rollback = errReply != nil
|
||||
if !rollback {
|
||||
return protocol.MakeIntReply(1)
|
||||
}
|
||||
return errReply
|
||||
}
|
||||
|
||||
func prepareMSetNx(cluster *Cluster, conn redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
args := cmdLine[1:]
|
||||
if len(args)%2 != 0 {
|
||||
return protocol.MakeSyntaxErrReply()
|
||||
}
|
||||
size := len(args) / 2
|
||||
values := make([][]byte, size)
|
||||
keys := make([]string, size)
|
||||
for i := 0; i < size; i++ {
|
||||
keys[i] = string(args[2*i])
|
||||
values[i] = args[2*i+1]
|
||||
}
|
||||
re := cluster.db.ExecWithLock(conn, utils.ToCmdLine2("ExistIn", keys...))
|
||||
if protocol.IsErrorReply(re) {
|
||||
return re
|
||||
}
|
||||
_, ok := re.(*protocol.EmptyMultiBulkReply)
|
||||
if !ok {
|
||||
return protocol.MakeErrReply(keyExistsErr)
|
||||
}
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
|
||||
func init() {
|
||||
registerPrepareFunc("MSetNx", prepareMSetNx)
|
||||
}
|
||||
@@ -1,30 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMSet(t *testing.T) {
|
||||
conn := connection.NewFakeConn()
|
||||
allowFastTransaction = false
|
||||
testNodeA := testCluster[0]
|
||||
ret := MSet(testNodeA, conn, toArgs("MSET", "a", "a", "b", "b"))
|
||||
asserts.AssertNotError(t, ret)
|
||||
ret = testNodeA.Exec(conn, toArgs("MGET", "a", "b"))
|
||||
asserts.AssertMultiBulkReply(t, ret, []string{"a", "b"})
|
||||
}
|
||||
|
||||
func TestMSetNx(t *testing.T) {
|
||||
conn := connection.NewFakeConn()
|
||||
allowFastTransaction = false
|
||||
testNodeA := testCluster[0]
|
||||
FlushAll(testNodeA, conn, toArgs("FLUSHALL"))
|
||||
ret := MSetNX(testNodeA, conn, toArgs("MSETNX", "a", "a", "b", "b"))
|
||||
asserts.AssertNotError(t, ret)
|
||||
ret = MSetNX(testNodeA, conn, toArgs("MSETNX", "a", "a", "c", "c"))
|
||||
asserts.AssertNotError(t, ret)
|
||||
ret = testNodeA.Exec(conn, toArgs("MGET", "a", "b", "c"))
|
||||
asserts.AssertMultiBulkReply(t, ret, []string{"a", "b", ""})
|
||||
}
|
||||
165
cluster/multi.go
165
cluster/multi.go
@@ -1,165 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/database"
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"strconv"
|
||||
)
|
||||
|
||||
const relayMulti = "multi_"
|
||||
const innerWatch = "watch_"
|
||||
|
||||
var relayMultiBytes = []byte(relayMulti)
|
||||
|
||||
// cmdLine == []string{"exec"}
|
||||
func execMulti(cluster *Cluster, conn redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if !conn.InMultiState() {
|
||||
return protocol.MakeErrReply("ERR EXEC without MULTI")
|
||||
}
|
||||
defer conn.SetMultiState(false)
|
||||
cmdLines := conn.GetQueuedCmdLine()
|
||||
|
||||
// analysis related keys
|
||||
keys := make([]string, 0) // may contains duplicate
|
||||
for _, cl := range cmdLines {
|
||||
wKeys, rKeys := database.GetRelatedKeys(cl)
|
||||
keys = append(keys, wKeys...)
|
||||
keys = append(keys, rKeys...)
|
||||
}
|
||||
watching := conn.GetWatching()
|
||||
watchingKeys := make([]string, 0, len(watching))
|
||||
for key := range watching {
|
||||
watchingKeys = append(watchingKeys, key)
|
||||
}
|
||||
keys = append(keys, watchingKeys...)
|
||||
if len(keys) == 0 {
|
||||
// empty transaction or only `PING`s
|
||||
return cluster.db.ExecMulti(conn, watching, cmdLines)
|
||||
}
|
||||
groupMap := cluster.groupBy(keys)
|
||||
if len(groupMap) > 1 {
|
||||
return protocol.MakeErrReply("ERR MULTI commands transaction must within one slot in cluster mode")
|
||||
}
|
||||
var peer string
|
||||
// assert len(groupMap) == 1
|
||||
for p := range groupMap {
|
||||
peer = p
|
||||
}
|
||||
|
||||
// out parser not support protocol.MultiRawReply, so we have to encode it
|
||||
if peer == cluster.self {
|
||||
for _, key := range keys {
|
||||
if errReply := cluster.ensureKey(key); errReply != nil {
|
||||
return errReply
|
||||
}
|
||||
}
|
||||
return cluster.db.ExecMulti(conn, watching, cmdLines)
|
||||
}
|
||||
return execMultiOnOtherNode(cluster, conn, peer, watching, cmdLines)
|
||||
}
|
||||
|
||||
func execMultiOnOtherNode(cluster *Cluster, conn redis.Connection, peer string, watching map[string]uint32, cmdLines []CmdLine) redis.Reply {
|
||||
defer func() {
|
||||
conn.ClearQueuedCmds()
|
||||
conn.SetMultiState(false)
|
||||
}()
|
||||
relayCmdLine := [][]byte{ // relay it to executing node
|
||||
relayMultiBytes,
|
||||
}
|
||||
// watching commands
|
||||
var watchingCmdLine = utils.ToCmdLine(innerWatch)
|
||||
for key, ver := range watching {
|
||||
verStr := strconv.FormatUint(uint64(ver), 10)
|
||||
watchingCmdLine = append(watchingCmdLine, []byte(key), []byte(verStr))
|
||||
}
|
||||
relayCmdLine = append(relayCmdLine, encodeCmdLine([]CmdLine{watchingCmdLine})...)
|
||||
relayCmdLine = append(relayCmdLine, encodeCmdLine(cmdLines)...)
|
||||
var rawRelayResult redis.Reply
|
||||
rawRelayResult = cluster.relay(peer, connection.NewFakeConn(), relayCmdLine)
|
||||
if protocol.IsErrorReply(rawRelayResult) {
|
||||
return rawRelayResult
|
||||
}
|
||||
_, ok := rawRelayResult.(*protocol.EmptyMultiBulkReply)
|
||||
if ok {
|
||||
return rawRelayResult
|
||||
}
|
||||
relayResult, ok := rawRelayResult.(*protocol.MultiBulkReply)
|
||||
if !ok {
|
||||
return protocol.MakeErrReply("execute failed")
|
||||
}
|
||||
rep, err := parseEncodedMultiRawReply(relayResult.Args)
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply(err.Error())
|
||||
}
|
||||
return rep
|
||||
}
|
||||
|
||||
// execRelayedMulti execute relayed multi commands transaction
|
||||
// cmdLine format: _multi watch-cmdLine base64ed-cmdLine
|
||||
// result format: base64ed-protocol list
|
||||
func execRelayedMulti(cluster *Cluster, conn redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if len(cmdLine) < 2 {
|
||||
return protocol.MakeArgNumErrReply("_exec")
|
||||
}
|
||||
decoded, err := parseEncodedMultiRawReply(cmdLine[1:])
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply(err.Error())
|
||||
}
|
||||
var txCmdLines []CmdLine
|
||||
for _, rep := range decoded.Replies {
|
||||
mbr, ok := rep.(*protocol.MultiBulkReply)
|
||||
if !ok {
|
||||
return protocol.MakeErrReply("exec failed")
|
||||
}
|
||||
txCmdLines = append(txCmdLines, mbr.Args)
|
||||
}
|
||||
watching := make(map[string]uint32)
|
||||
watchCmdLine := txCmdLines[0] // format: watch_ key1 ver1 key2 ver2...
|
||||
for i := 2; i < len(watchCmdLine); i += 2 {
|
||||
key := string(watchCmdLine[i-1])
|
||||
verStr := string(watchCmdLine[i])
|
||||
ver, err := strconv.ParseUint(verStr, 10, 64)
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply("watching command line failed")
|
||||
}
|
||||
watching[key] = uint32(ver)
|
||||
}
|
||||
rawResult := cluster.db.ExecMulti(conn, watching, txCmdLines[1:])
|
||||
_, ok := rawResult.(*protocol.EmptyMultiBulkReply)
|
||||
if ok {
|
||||
return rawResult
|
||||
}
|
||||
resultMBR, ok := rawResult.(*protocol.MultiRawReply)
|
||||
if !ok {
|
||||
return protocol.MakeErrReply("exec failed")
|
||||
}
|
||||
return encodeMultiRawReply(resultMBR)
|
||||
}
|
||||
|
||||
func execWatch(cluster *Cluster, conn redis.Connection, args [][]byte) redis.Reply {
|
||||
if len(args) < 2 {
|
||||
return protocol.MakeArgNumErrReply("watch")
|
||||
}
|
||||
args = args[1:]
|
||||
watching := conn.GetWatching()
|
||||
for _, bkey := range args {
|
||||
key := string(bkey)
|
||||
err := cluster.ensureKey(key)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
result := cluster.relayByKey(key, conn, utils.ToCmdLine("GetVer", key))
|
||||
if protocol.IsErrorReply(result) {
|
||||
return result
|
||||
}
|
||||
intResult, ok := result.(*protocol.IntReply)
|
||||
if !ok {
|
||||
return protocol.MakeErrReply("get version failed")
|
||||
}
|
||||
watching[key] = uint32(intResult.Code)
|
||||
}
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
@@ -1,48 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/base64"
|
||||
"github.com/hdt3213/godis/redis/parser"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
)
|
||||
|
||||
func encodeCmdLine(cmdLines []CmdLine) [][]byte {
|
||||
var result [][]byte
|
||||
for _, line := range cmdLines {
|
||||
raw := protocol.MakeMultiBulkReply(line).ToBytes()
|
||||
encoded := make([]byte, base64.StdEncoding.EncodedLen(len(raw)))
|
||||
base64.StdEncoding.Encode(encoded, raw)
|
||||
result = append(result, encoded)
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
func parseEncodedMultiRawReply(args [][]byte) (*protocol.MultiRawReply, error) {
|
||||
cmdBuf := new(bytes.Buffer)
|
||||
for _, arg := range args {
|
||||
dbuf := make([]byte, base64.StdEncoding.DecodedLen(len(arg)))
|
||||
n, err := base64.StdEncoding.Decode(dbuf, arg)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
cmdBuf.Write(dbuf[:n])
|
||||
}
|
||||
cmds, err := parser.ParseBytes(cmdBuf.Bytes())
|
||||
if err != nil {
|
||||
return nil, protocol.MakeErrReply(err.Error())
|
||||
}
|
||||
return protocol.MakeMultiRawReply(cmds), nil
|
||||
}
|
||||
|
||||
// todo: use multi raw reply instead of base64
|
||||
func encodeMultiRawReply(src *protocol.MultiRawReply) *protocol.MultiBulkReply {
|
||||
args := make([][]byte, 0, len(src.Replies))
|
||||
for _, rep := range src.Replies {
|
||||
raw := rep.ToBytes()
|
||||
encoded := make([]byte, base64.StdEncoding.EncodedLen(len(raw)))
|
||||
base64.StdEncoding.Encode(encoded, raw)
|
||||
args = append(args, encoded)
|
||||
}
|
||||
return protocol.MakeMultiBulkReply(args)
|
||||
}
|
||||
@@ -1,92 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMultiExecOnSelf(t *testing.T) {
|
||||
testNodeA := testCluster[0]
|
||||
conn := new(connection.FakeConn)
|
||||
testNodeA.db.Exec(conn, utils.ToCmdLine("FLUSHALL"))
|
||||
result := testNodeA.Exec(conn, toArgs("MULTI"))
|
||||
asserts.AssertNotError(t, result)
|
||||
key := "{abc}" + utils.RandString(10)
|
||||
value := utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("set", key, value))
|
||||
key2 := "{abc}" + utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("rpush", key2, value))
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("exec"))
|
||||
asserts.AssertNotError(t, result)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("get", key))
|
||||
asserts.AssertBulkReply(t, result, value)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("lrange", key2, "0", "-1"))
|
||||
asserts.AssertMultiBulkReply(t, result, []string{value})
|
||||
}
|
||||
|
||||
func TestEmptyMulti(t *testing.T) {
|
||||
testNodeA := testCluster[0]
|
||||
conn := new(connection.FakeConn)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("FLUSHALL"))
|
||||
result := testNodeA.Exec(conn, toArgs("MULTI"))
|
||||
asserts.AssertNotError(t, result)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("GET", "a"))
|
||||
asserts.AssertNotError(t, result)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("EXEC"))
|
||||
asserts.AssertNotError(t, result)
|
||||
mbr := result.(*protocol.MultiRawReply)
|
||||
asserts.AssertNullBulk(t, mbr.Replies[0])
|
||||
}
|
||||
|
||||
func TestMultiExecOnOthers(t *testing.T) {
|
||||
testNodeA := testCluster[0]
|
||||
conn := new(connection.FakeConn)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("FLUSHALL"))
|
||||
result := testNodeA.Exec(conn, toArgs("MULTI"))
|
||||
asserts.AssertNotError(t, result)
|
||||
key := utils.RandString(10)
|
||||
value := utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("rpush", key, value))
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("lrange", key, "0", "-1"))
|
||||
|
||||
cmdLines := conn.GetQueuedCmdLine()
|
||||
rawResp := execMultiOnOtherNode(testNodeA, conn, testNodeA.self, nil, cmdLines)
|
||||
rep := rawResp.(*protocol.MultiRawReply)
|
||||
if len(rep.Replies) != 2 {
|
||||
t.Errorf("expect 2 replies actual %d", len(rep.Replies))
|
||||
}
|
||||
asserts.AssertMultiBulkReply(t, rep.Replies[1], []string{value})
|
||||
}
|
||||
|
||||
func TestWatch(t *testing.T) {
|
||||
testNodeA := testCluster[0]
|
||||
for i := 0; i < 10; i++ {
|
||||
conn := new(connection.FakeConn)
|
||||
key := "{1}" + utils.RandString(10)
|
||||
key2 := "{1}" + utils.RandString(10) // use hash tag to ensure same slot
|
||||
value := utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("FLUSHALL"))
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("watch", key))
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("set", key, value))
|
||||
result := testNodeA.Exec(conn, toArgs("MULTI"))
|
||||
asserts.AssertNotError(t, result)
|
||||
value2 := utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("set", key2, value2))
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("exec"))
|
||||
asserts.AssertNotError(t, result)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("get", key2))
|
||||
asserts.AssertNullBulk(t, result)
|
||||
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("watch", key))
|
||||
result = testNodeA.Exec(conn, toArgs("MULTI"))
|
||||
asserts.AssertNotError(t, result)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("set", key2, value2))
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("exec"))
|
||||
asserts.AssertNotError(t, result)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("get", key2))
|
||||
asserts.AssertBulkReply(t, result, value2)
|
||||
}
|
||||
}
|
||||
@@ -1,35 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/logger"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
)
|
||||
|
||||
const (
|
||||
relayPublish = "publish_"
|
||||
)
|
||||
|
||||
// Publish broadcasts msg to all peers in cluster when receive publish command from client
|
||||
func Publish(cluster *Cluster, c redis.Connection, cmdLine [][]byte) redis.Reply {
|
||||
var count int64 = 0
|
||||
results := cluster.broadcast(c, modifyCmd(cmdLine, relayPublish))
|
||||
for _, val := range results {
|
||||
if errReply, ok := val.(protocol.ErrorReply); ok {
|
||||
logger.Error("publish occurs error: " + errReply.Error())
|
||||
} else if intReply, ok := val.(*protocol.IntReply); ok {
|
||||
count += intReply.Code
|
||||
}
|
||||
}
|
||||
return protocol.MakeIntReply(count)
|
||||
}
|
||||
|
||||
// Subscribe puts the given connection into the given channel
|
||||
func Subscribe(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
return cluster.db.Exec(c, args) // let local db.hub handle subscribe
|
||||
}
|
||||
|
||||
// UnSubscribe removes the given connection from the given channel
|
||||
func UnSubscribe(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
return cluster.db.Exec(c, args) // let local db.hub handle subscribe
|
||||
}
|
||||
@@ -1,49 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/parser"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestPublish(t *testing.T) {
|
||||
testNodeA := testCluster[0]
|
||||
channel := utils.RandString(5)
|
||||
msg := utils.RandString(5)
|
||||
conn := connection.NewFakeConn()
|
||||
Subscribe(testNodeA, conn, utils.ToCmdLine("SUBSCRIBE", channel))
|
||||
conn.Clean() // clean subscribe success
|
||||
Publish(testNodeA, conn, utils.ToCmdLine("PUBLISH", channel, msg))
|
||||
data := conn.Bytes()
|
||||
ret, err := parser.ParseOne(data)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
}
|
||||
asserts.AssertMultiBulkReply(t, ret, []string{
|
||||
"message",
|
||||
channel,
|
||||
msg,
|
||||
})
|
||||
|
||||
// unsubscribe
|
||||
UnSubscribe(testNodeA, conn, utils.ToCmdLine("UNSUBSCRIBE", channel))
|
||||
conn.Clean()
|
||||
Publish(testNodeA, conn, utils.ToCmdLine("PUBLISH", channel, msg))
|
||||
data = conn.Bytes()
|
||||
if len(data) > 0 {
|
||||
t.Error("expect no msg")
|
||||
}
|
||||
|
||||
// unsubscribe all
|
||||
Subscribe(testNodeA, conn, utils.ToCmdLine("SUBSCRIBE", channel))
|
||||
UnSubscribe(testNodeA, conn, utils.ToCmdLine("UNSUBSCRIBE"))
|
||||
conn.Clean()
|
||||
Publish(testNodeA, conn, utils.ToCmdLine("PUBLISH", channel, msg))
|
||||
data = conn.Bytes()
|
||||
if len(data) > 0 {
|
||||
t.Error("expect no msg")
|
||||
}
|
||||
}
|
||||
1075
cluster/raft.go
1075
cluster/raft.go
File diff suppressed because it is too large
Load Diff
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)
|
||||
}
|
||||
194
cluster/raft/raft.go
Normal file
194
cluster/raft/raft.go
Normal file
@@ -0,0 +1,194 @@
|
||||
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
|
||||
}
|
||||
|
||||
func (node *Node) Close() error {
|
||||
future := node.inner.Shutdown()
|
||||
return fmt.Errorf("raft shutdown %v", future.Error())
|
||||
}
|
||||
@@ -1,134 +0,0 @@
|
||||
package cluster
|
||||
|
||||
// raft event handlers
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"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 (
|
||||
eventNewNode = iota + 1
|
||||
eventSetSlot
|
||||
)
|
||||
|
||||
// invoker should provide with raft.mu lock
|
||||
func (raft *Raft) applyLogEntries(entries []*logEntry) {
|
||||
for _, entry := range entries {
|
||||
switch entry.Event {
|
||||
case eventNewNode:
|
||||
node := &Node{
|
||||
ID: entry.NodeID,
|
||||
Addr: entry.Addr,
|
||||
}
|
||||
raft.nodes[node.ID] = node
|
||||
if raft.state == leader {
|
||||
raft.nodeIndexMap[entry.NodeID] = &nodeStatus{
|
||||
receivedIndex: entry.Index, // the new node should not receive its own join event
|
||||
}
|
||||
}
|
||||
case eventSetSlot:
|
||||
for _, slotID := range entry.SlotIDs {
|
||||
slot := raft.slots[slotID]
|
||||
oldNode := raft.nodes[slot.NodeID]
|
||||
// remove from old oldNode
|
||||
for i, s := range oldNode.Slots {
|
||||
if s.ID == slot.ID {
|
||||
copy(oldNode.Slots[i:], oldNode.Slots[i+1:])
|
||||
oldNode.Slots = oldNode.Slots[:len(oldNode.Slots)-1]
|
||||
break
|
||||
}
|
||||
}
|
||||
newNodeID := entry.NodeID
|
||||
slot.NodeID = newNodeID
|
||||
// fixme: 多个节点同时加入后 re balance 时 newNode 可能为 nil
|
||||
newNode := raft.nodes[slot.NodeID]
|
||||
newNode.Slots = append(newNode.Slots, slot)
|
||||
}
|
||||
}
|
||||
}
|
||||
if err := raft.persist(); err != nil {
|
||||
logger.Errorf("persist raft error: %v", err)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// NewNode creates a new Node when a node request self node for joining cluster
|
||||
func (raft *Raft) NewNode(addr string) (*Node, error) {
|
||||
if _, ok := raft.nodes[addr]; ok {
|
||||
return nil, errors.New("node existed")
|
||||
}
|
||||
node := &Node{
|
||||
ID: addr,
|
||||
Addr: addr,
|
||||
}
|
||||
raft.nodes[node.ID] = node
|
||||
proposal := &logEntry{
|
||||
Event: eventNewNode,
|
||||
NodeID: node.ID,
|
||||
Addr: node.Addr,
|
||||
}
|
||||
conn := connection.NewFakeConn()
|
||||
resp := raft.cluster.relay(raft.leaderId, conn,
|
||||
utils.ToCmdLine("raft", "propose", string(proposal.marshal())))
|
||||
if err, ok := resp.(protocol.ErrorReply); ok {
|
||||
return nil, err
|
||||
}
|
||||
return node, nil
|
||||
}
|
||||
|
||||
// SetSlot propose
|
||||
func (raft *Raft) SetSlot(slotIDs []uint32, newNodeID string) protocol.ErrorReply {
|
||||
proposal := &logEntry{
|
||||
Event: eventSetSlot,
|
||||
NodeID: newNodeID,
|
||||
SlotIDs: slotIDs,
|
||||
}
|
||||
conn := connection.NewFakeConn()
|
||||
resp := raft.cluster.relay(raft.leaderId, conn,
|
||||
utils.ToCmdLine("raft", "propose", string(proposal.marshal())))
|
||||
if err, ok := resp.(protocol.ErrorReply); ok {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// execRaftJoin handles requests from a new node to join raft group, current node should be leader
|
||||
// command line: raft join addr
|
||||
func execRaftJoin(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
if len(args) != 1 {
|
||||
return protocol.MakeArgNumErrReply("raft join")
|
||||
}
|
||||
raft := cluster.asRaft()
|
||||
if raft.state != leader {
|
||||
leaderNode := raft.nodes[raft.leaderId]
|
||||
return protocol.MakeErrReply("NOT LEADER " + leaderNode.ID + " " + leaderNode.Addr)
|
||||
}
|
||||
addr := string(args[0])
|
||||
nodeID := addr
|
||||
|
||||
raft.mu.RLock()
|
||||
_, exist := raft.nodes[addr]
|
||||
raft.mu.RUnlock()
|
||||
// if node has joint cluster but terminated before persisting cluster config,
|
||||
// it may try to join at next start.
|
||||
// In this case, we only have to send a snapshot for it
|
||||
if !exist {
|
||||
proposal := &logEntry{
|
||||
Event: eventNewNode,
|
||||
NodeID: nodeID,
|
||||
Addr: addr,
|
||||
}
|
||||
if err := raft.propose(proposal); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
raft.mu.RLock()
|
||||
snapshot := raft.makeSnapshotForFollower(nodeID)
|
||||
raft.mu.RUnlock()
|
||||
return protocol.MakeMultiBulkReply(snapshot)
|
||||
}
|
||||
@@ -1,204 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
)
|
||||
|
||||
// marshalSlotIds serializes slot ids
|
||||
// For example, 1, 2, 3, 5, 7, 8 -> 1-3, 5, 7-8
|
||||
func marshalSlotIds(slots []*Slot) []string {
|
||||
sort.Slice(slots, func(i, j int) bool {
|
||||
return slots[i].ID < slots[j].ID
|
||||
})
|
||||
// find continuous scopes
|
||||
var scopes [][]uint32
|
||||
buf := make([]uint32, 2)
|
||||
var scope []uint32
|
||||
for i, slot := range slots {
|
||||
if len(scope) == 0 { // outside scope
|
||||
if i+1 < len(slots) &&
|
||||
slots[i+1].ID == slot.ID+1 { // if continuous, then start one
|
||||
scope = buf
|
||||
scope[0] = slot.ID
|
||||
} else { // discrete number
|
||||
scopes = append(scopes, []uint32{slot.ID})
|
||||
}
|
||||
} else { // within a scope
|
||||
if i == len(slots)-1 || slots[i+1].ID != slot.ID+1 { // reach end or not continuous, stop current scope
|
||||
scope[1] = slot.ID
|
||||
scopes = append(scopes, []uint32{scope[0], scope[1]})
|
||||
scope = nil
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// marshal scopes
|
||||
result := make([]string, 0, len(scopes))
|
||||
for _, scope := range scopes {
|
||||
if len(scope) == 1 {
|
||||
s := strconv.Itoa(int(scope[0]))
|
||||
result = append(result, s)
|
||||
} else { // assert len(scope) == 2
|
||||
beg := strconv.Itoa(int(scope[0]))
|
||||
end := strconv.Itoa(int(scope[1]))
|
||||
result = append(result, beg+"-"+end)
|
||||
}
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
// unmarshalSlotIds deserializes lines generated by marshalSlotIds
|
||||
func unmarshalSlotIds(args []string) ([]uint32, error) {
|
||||
var result []uint32
|
||||
for i, line := range args {
|
||||
if pivot := strings.IndexByte(line, '-'); pivot > 0 {
|
||||
// line is a scope
|
||||
beg, err := strconv.Atoi(line[:pivot])
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("illegal at slot line %d", i+1)
|
||||
}
|
||||
end, err := strconv.Atoi(line[pivot+1:])
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("illegal at slot line %d", i+1)
|
||||
}
|
||||
for j := beg; j <= end; j++ {
|
||||
result = append(result, uint32(j))
|
||||
}
|
||||
} else {
|
||||
// line is a number
|
||||
v, err := strconv.Atoi(line)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("illegal at slot line %d", i)
|
||||
}
|
||||
result = append(result, uint32(v))
|
||||
}
|
||||
}
|
||||
return result, nil
|
||||
}
|
||||
|
||||
type nodePayload struct {
|
||||
ID string `json:"id"`
|
||||
Addr string `json:"addr"`
|
||||
SlotDesc []string `json:"slotDesc"`
|
||||
Flags uint32 `json:"flags"`
|
||||
}
|
||||
|
||||
func marshalNodes(nodes map[string]*Node) [][]byte {
|
||||
var args [][]byte
|
||||
for _, node := range nodes {
|
||||
slotLines := marshalSlotIds(node.Slots)
|
||||
payload := &nodePayload{
|
||||
ID: node.ID,
|
||||
Addr: node.Addr,
|
||||
SlotDesc: slotLines,
|
||||
Flags: node.Flags,
|
||||
}
|
||||
bin, _ := json.Marshal(payload)
|
||||
args = append(args, bin)
|
||||
}
|
||||
return args
|
||||
}
|
||||
|
||||
func unmarshalNodes(args [][]byte) (map[string]*Node, error) {
|
||||
nodeMap := make(map[string]*Node)
|
||||
for i, bin := range args {
|
||||
payload := &nodePayload{}
|
||||
err := json.Unmarshal(bin, payload)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("unmarshal node failed at line %d: %v", i+1, err)
|
||||
}
|
||||
slotIds, err := unmarshalSlotIds(payload.SlotDesc)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
node := &Node{
|
||||
ID: payload.ID,
|
||||
Addr: payload.Addr,
|
||||
Flags: payload.Flags,
|
||||
}
|
||||
for _, slotId := range slotIds {
|
||||
node.Slots = append(node.Slots, &Slot{
|
||||
ID: slotId,
|
||||
NodeID: node.ID,
|
||||
Flags: 0,
|
||||
})
|
||||
}
|
||||
nodeMap[node.ID] = node
|
||||
}
|
||||
return nodeMap, nil
|
||||
}
|
||||
|
||||
// genSnapshot
|
||||
// invoker provide lock
|
||||
func (raft *Raft) makeSnapshot() [][]byte {
|
||||
topology := marshalNodes(raft.nodes)
|
||||
snapshot := [][]byte{
|
||||
[]byte(raft.selfNodeID),
|
||||
[]byte(strconv.Itoa(int(raft.state))),
|
||||
[]byte(raft.leaderId),
|
||||
[]byte(strconv.Itoa(raft.term)),
|
||||
[]byte(strconv.Itoa(raft.committedIndex)),
|
||||
}
|
||||
snapshot = append(snapshot, topology...)
|
||||
return snapshot
|
||||
}
|
||||
|
||||
// makeSnapshotForFollower used by leader node to generate snapshot for follower
|
||||
// invoker provide with lock
|
||||
func (raft *Raft) makeSnapshotForFollower(followerId string) [][]byte {
|
||||
snapshot := raft.makeSnapshot()
|
||||
snapshot[0] = []byte(followerId)
|
||||
snapshot[1] = []byte(strconv.Itoa(int(follower)))
|
||||
return snapshot
|
||||
}
|
||||
|
||||
// invoker provide with lock
|
||||
func (raft *Raft) loadSnapshot(snapshot [][]byte) protocol.ErrorReply {
|
||||
// make sure raft.slots and node.Slots is the same object
|
||||
selfNodeId := string(snapshot[0])
|
||||
state0, err := strconv.Atoi(string(snapshot[1]))
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply("illegal state: " + string(snapshot[1]))
|
||||
}
|
||||
state := raftState(state0)
|
||||
if _, ok := stateNames[state]; !ok {
|
||||
return protocol.MakeErrReply("unknown state: " + strconv.Itoa(int(state)))
|
||||
}
|
||||
leaderId := string(snapshot[2])
|
||||
term, err := strconv.Atoi(string(snapshot[3]))
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply("illegal term: " + string(snapshot[3]))
|
||||
}
|
||||
commitIndex, err := strconv.Atoi(string(snapshot[4]))
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply("illegal commit index: " + string(snapshot[3]))
|
||||
}
|
||||
nodes, err := unmarshalNodes(snapshot[5:])
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply(err.Error())
|
||||
}
|
||||
raft.selfNodeID = selfNodeId
|
||||
raft.state = state
|
||||
raft.leaderId = leaderId
|
||||
raft.term = term
|
||||
raft.committedIndex = commitIndex
|
||||
raft.proposedIndex = commitIndex
|
||||
raft.initLog(term, commitIndex, nil)
|
||||
raft.slots = make([]*Slot, slotCount)
|
||||
for _, node := range nodes {
|
||||
for _, slot := range node.Slots {
|
||||
raft.slots[int(slot.ID)] = slot
|
||||
}
|
||||
if node.getState() == leader {
|
||||
raft.leaderId = node.ID
|
||||
}
|
||||
}
|
||||
raft.nodes = nodes
|
||||
return nil
|
||||
}
|
||||
@@ -1 +0,0 @@
|
||||
package cluster
|
||||
@@ -1,140 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"strconv"
|
||||
)
|
||||
|
||||
// Rename renames a key, the origin and the destination must within the same node
|
||||
func Rename(cluster *Cluster, c redis.Connection, cmdLine [][]byte) redis.Reply {
|
||||
if len(cmdLine) != 3 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'rename' command")
|
||||
}
|
||||
srcKey := string(cmdLine[1])
|
||||
destKey := string(cmdLine[2])
|
||||
srcNode := cluster.pickNodeAddrByKey(srcKey)
|
||||
destNode := cluster.pickNodeAddrByKey(destKey)
|
||||
if srcNode == destNode { // do fast
|
||||
return cluster.relay(srcNode, c, modifyCmd(cmdLine, "Rename_"))
|
||||
}
|
||||
groupMap := map[string][]string{
|
||||
srcNode: {srcKey},
|
||||
destNode: {destKey},
|
||||
}
|
||||
txID := cluster.idGenerator.NextID()
|
||||
txIDStr := strconv.FormatInt(txID, 10)
|
||||
// prepare rename from
|
||||
srcPrepareResp := cluster.relay(srcNode, c, makeArgs("Prepare", txIDStr, "RenameFrom", srcKey))
|
||||
if protocol.IsErrorReply(srcPrepareResp) {
|
||||
// rollback src node
|
||||
requestRollback(cluster, c, txID, map[string][]string{srcNode: {srcKey}})
|
||||
return srcPrepareResp
|
||||
}
|
||||
srcPrepareMBR, ok := srcPrepareResp.(*protocol.MultiBulkReply)
|
||||
if !ok || len(srcPrepareMBR.Args) < 2 {
|
||||
requestRollback(cluster, c, txID, map[string][]string{srcNode: {srcKey}})
|
||||
return protocol.MakeErrReply("ERR invalid prepare response")
|
||||
}
|
||||
// prepare rename to
|
||||
destPrepareResp := cluster.relay(destNode, c, utils.ToCmdLine3("Prepare", []byte(txIDStr),
|
||||
[]byte("RenameTo"), []byte(destKey), srcPrepareMBR.Args[0], srcPrepareMBR.Args[1]))
|
||||
if protocol.IsErrorReply(destPrepareResp) {
|
||||
// rollback src node
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
return destPrepareResp
|
||||
}
|
||||
if _, errReply := requestCommit(cluster, c, txID, groupMap); errReply != nil {
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
return errReply
|
||||
}
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
|
||||
// prepareRenameFrom is prepare-function for RenameFrom, see prepareFuncMap
|
||||
func prepareRenameFrom(cluster *Cluster, conn redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if len(cmdLine) != 2 {
|
||||
return protocol.MakeArgNumErrReply("RenameFrom")
|
||||
}
|
||||
key := string(cmdLine[1])
|
||||
existResp := cluster.db.ExecWithLock(conn, utils.ToCmdLine("Exists", key))
|
||||
if protocol.IsErrorReply(existResp) {
|
||||
return existResp
|
||||
}
|
||||
existIntResp := existResp.(*protocol.IntReply)
|
||||
if existIntResp.Code == 0 {
|
||||
return protocol.MakeErrReply("ERR no such key")
|
||||
}
|
||||
return cluster.db.ExecWithLock(conn, utils.ToCmdLine2("DumpKey", key))
|
||||
}
|
||||
|
||||
func prepareRenameNxTo(cluster *Cluster, conn redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if len(cmdLine) != 4 {
|
||||
return protocol.MakeArgNumErrReply("RenameNxTo")
|
||||
}
|
||||
key := string(cmdLine[1])
|
||||
existResp := cluster.db.ExecWithLock(conn, utils.ToCmdLine("Exists", key))
|
||||
if protocol.IsErrorReply(existResp) {
|
||||
return existResp
|
||||
}
|
||||
existIntResp := existResp.(*protocol.IntReply)
|
||||
if existIntResp.Code == 1 {
|
||||
return protocol.MakeErrReply(keyExistsErr)
|
||||
}
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
|
||||
func init() {
|
||||
registerPrepareFunc("RenameFrom", prepareRenameFrom)
|
||||
registerPrepareFunc("RenameNxTo", prepareRenameNxTo)
|
||||
}
|
||||
|
||||
// RenameNx renames a key, only if the new key does not exist.
|
||||
// The origin and the destination must within the same node
|
||||
func RenameNx(cluster *Cluster, c redis.Connection, cmdLine [][]byte) redis.Reply {
|
||||
if len(cmdLine) != 3 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'renamenx' command")
|
||||
}
|
||||
srcKey := string(cmdLine[1])
|
||||
destKey := string(cmdLine[2])
|
||||
srcNode := cluster.pickNodeAddrByKey(srcKey)
|
||||
destNode := cluster.pickNodeAddrByKey(destKey)
|
||||
if srcNode == destNode {
|
||||
return cluster.relay(srcNode, c, modifyCmd(cmdLine, "RenameNX_"))
|
||||
}
|
||||
groupMap := map[string][]string{
|
||||
srcNode: {srcKey},
|
||||
destNode: {destKey},
|
||||
}
|
||||
txID := cluster.idGenerator.NextID()
|
||||
txIDStr := strconv.FormatInt(txID, 10)
|
||||
// prepare rename from
|
||||
srcPrepareResp := cluster.relay(srcNode, c, makeArgs("Prepare", txIDStr, "RenameFrom", srcKey))
|
||||
if protocol.IsErrorReply(srcPrepareResp) {
|
||||
// rollback src node
|
||||
requestRollback(cluster, c, txID, map[string][]string{srcNode: {srcKey}})
|
||||
return srcPrepareResp
|
||||
}
|
||||
srcPrepareMBR, ok := srcPrepareResp.(*protocol.MultiBulkReply)
|
||||
if !ok || len(srcPrepareMBR.Args) < 2 {
|
||||
requestRollback(cluster, c, txID, map[string][]string{srcNode: {srcKey}})
|
||||
return protocol.MakeErrReply("ERR invalid prepare response")
|
||||
}
|
||||
// prepare rename to
|
||||
destPrepareResp := cluster.relay(destNode, c, utils.ToCmdLine3("Prepare", []byte(txIDStr),
|
||||
[]byte("RenameNxTo"), []byte(destKey), srcPrepareMBR.Args[0], srcPrepareMBR.Args[1]))
|
||||
if protocol.IsErrorReply(destPrepareResp) {
|
||||
// rollback src node
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
if re := destPrepareResp.(protocol.ErrorReply); re.Error() == keyExistsErr {
|
||||
return protocol.MakeIntReply(0)
|
||||
}
|
||||
return destPrepareResp
|
||||
}
|
||||
if _, errReply := requestCommit(cluster, c, txID, groupMap); errReply != nil {
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
return errReply
|
||||
}
|
||||
return protocol.MakeIntReply(1)
|
||||
}
|
||||
@@ -1,59 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestRename(t *testing.T) {
|
||||
testNodeA := testCluster[0]
|
||||
conn := new(connection.FakeConn)
|
||||
|
||||
// cross node rename
|
||||
for i := 0; i < 10; i++ {
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("FlushALL"))
|
||||
key := utils.RandString(10)
|
||||
value := utils.RandString(10)
|
||||
newKey := utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", key, value, "ex", "100000"))
|
||||
result := testNodeA.Exec(conn, utils.ToCmdLine("RENAME", key, newKey))
|
||||
asserts.AssertStatusReply(t, result, "OK")
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("EXISTS", key))
|
||||
asserts.AssertIntReply(t, result, 0)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("EXISTS", newKey))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("TTL", newKey))
|
||||
asserts.AssertIntReplyGreaterThan(t, result, 0)
|
||||
}
|
||||
}
|
||||
|
||||
func TestRenameNx(t *testing.T) {
|
||||
testNodeA := testCluster[0]
|
||||
conn := new(connection.FakeConn)
|
||||
|
||||
// cross node rename
|
||||
for i := 0; i < 10; i++ {
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("FlushALL"))
|
||||
key := utils.RandString(10)
|
||||
value := utils.RandString(10)
|
||||
newKey := utils.RandString(10)
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", key, value, "ex", "100000"))
|
||||
result := testNodeA.Exec(conn, utils.ToCmdLine("RENAMENX", key, newKey))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("EXISTS", key))
|
||||
asserts.AssertIntReply(t, result, 0)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("EXISTS", newKey))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("TTL", newKey))
|
||||
asserts.AssertIntReplyGreaterThan(t, result, 0)
|
||||
|
||||
value2 := value + "ccc"
|
||||
testNodeA.Exec(conn, utils.ToCmdLine("SET", key, value2, "ex", "100000"))
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("RENAMENX", key, newKey))
|
||||
asserts.AssertIntReply(t, result, 0)
|
||||
result = testNodeA.Exec(conn, utils.ToCmdLine("EXISTS", key))
|
||||
asserts.AssertIntReply(t, result, 1)
|
||||
}
|
||||
}
|
||||
@@ -1,166 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"strings"
|
||||
)
|
||||
|
||||
// CmdLine is alias for [][]byte, represents a command line
|
||||
type CmdLine = [][]byte
|
||||
|
||||
var router = make(map[string]CmdFunc)
|
||||
|
||||
func registerCmd(name string, cmd CmdFunc) {
|
||||
name = strings.ToLower(name)
|
||||
router[name] = cmd
|
||||
}
|
||||
|
||||
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.ID == cluster.self {
|
||||
err := cluster.ensureKeyWithoutLock(key)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// to self db
|
||||
//return cluster.db.Exec(c, cmdLine)
|
||||
return cluster.db.Exec(c, args)
|
||||
}
|
||||
return cluster.relay(peer.ID, c, args)
|
||||
}
|
||||
|
||||
func init() {
|
||||
registerCmd("Ping", ping)
|
||||
registerCmd("Prepare", execPrepare)
|
||||
registerCmd("Commit", execCommit)
|
||||
registerCmd("Rollback", execRollback)
|
||||
registerCmd("Del", Del)
|
||||
registerCmd("Rename", Rename)
|
||||
registerCmd("RenameNx", RenameNx)
|
||||
registerCmd("Copy", Copy)
|
||||
registerCmd("MSet", MSet)
|
||||
registerCmd("MGet", MGet)
|
||||
registerCmd("MSetNx", MSetNX)
|
||||
registerCmd("Publish", Publish)
|
||||
registerCmd("Subscribe", Subscribe)
|
||||
registerCmd("Unsubscribe", UnSubscribe)
|
||||
registerCmd("FlushDB", FlushDB)
|
||||
registerCmd("FlushAll", FlushAll)
|
||||
registerCmd(relayMulti, execRelayedMulti)
|
||||
registerCmd("Watch", execWatch)
|
||||
registerCmd("FlushDB_", genPenetratingExecutor("FlushDB"))
|
||||
registerCmd("Copy_", genPenetratingExecutor("Copy"))
|
||||
registerCmd("Watch_", genPenetratingExecutor("Watch"))
|
||||
registerCmd(relayPublish, genPenetratingExecutor("Publish"))
|
||||
registerCmd("Del_", genPenetratingExecutor("Del"))
|
||||
registerCmd("MSet_", genPenetratingExecutor("MSet"))
|
||||
registerCmd("MSetNx_", genPenetratingExecutor("MSetNx"))
|
||||
registerCmd("MGet_", genPenetratingExecutor("MGet"))
|
||||
registerCmd("Rename_", genPenetratingExecutor("Rename"))
|
||||
registerCmd("RenameNx_", genPenetratingExecutor("RenameNx"))
|
||||
registerCmd("DumpKey_", genPenetratingExecutor("DumpKey"))
|
||||
|
||||
defaultCmds := []string{
|
||||
"expire",
|
||||
"expireAt",
|
||||
"pExpire",
|
||||
"pExpireAt",
|
||||
"ttl",
|
||||
"PTtl",
|
||||
"persist",
|
||||
"exists",
|
||||
"type",
|
||||
"set",
|
||||
"setNx",
|
||||
"setEx",
|
||||
"pSetEx",
|
||||
"get",
|
||||
"getEx",
|
||||
"getSet",
|
||||
"getDel",
|
||||
"incr",
|
||||
"incrBy",
|
||||
"incrByFloat",
|
||||
"decr",
|
||||
"decrBy",
|
||||
"lPush",
|
||||
"lPushX",
|
||||
"rPush",
|
||||
"rPushX",
|
||||
"LPop",
|
||||
"RPop",
|
||||
"LRem",
|
||||
"LLen",
|
||||
"LIndex",
|
||||
"LSet",
|
||||
"LRange",
|
||||
"HSet",
|
||||
"HSetNx",
|
||||
"HGet",
|
||||
"HExists",
|
||||
"HDel",
|
||||
"HLen",
|
||||
"HStrLen",
|
||||
"HMGet",
|
||||
"HMSet",
|
||||
"HKeys",
|
||||
"HVals",
|
||||
"HGetAll",
|
||||
"HIncrBy",
|
||||
"HIncrByFloat",
|
||||
"HRandField",
|
||||
"SAdd",
|
||||
"SIsMember",
|
||||
"SRem",
|
||||
"SPop",
|
||||
"SCard",
|
||||
"SMembers",
|
||||
"SInter",
|
||||
"SInterStore",
|
||||
"SUnion",
|
||||
"SUnionStore",
|
||||
"SDiff",
|
||||
"SDiffStore",
|
||||
"SRandMember",
|
||||
"ZAdd",
|
||||
"ZScore",
|
||||
"ZIncrBy",
|
||||
"ZRank",
|
||||
"ZCount",
|
||||
"ZRevRank",
|
||||
"ZCard",
|
||||
"ZRange",
|
||||
"ZRevRange",
|
||||
"ZRangeByScore",
|
||||
"ZRevRangeByScore",
|
||||
"ZRem",
|
||||
"ZRemRangeByScore",
|
||||
"ZRemRangeByRank",
|
||||
"GeoAdd",
|
||||
"GeoPos",
|
||||
"GeoDist",
|
||||
"GeoHash",
|
||||
"GeoRadius",
|
||||
"GeoRadiusByMember",
|
||||
"GetVer",
|
||||
"DumpKey",
|
||||
}
|
||||
for _, name := range defaultCmds {
|
||||
registerDefaultCmd(name)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// genPenetratingExecutor generates an executor that can reach directly to the database layer
|
||||
func genPenetratingExecutor(realCmd string) CmdFunc {
|
||||
return func(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
return cluster.db.Exec(c, modifyCmd(cmdLine, realCmd))
|
||||
}
|
||||
}
|
||||
253
cluster/tcc.go
253
cluster/tcc.go
@@ -1,253 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/hdt3213/godis/database"
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/logger"
|
||||
"github.com/hdt3213/godis/lib/timewheel"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
// prepareFunc executed after related key locked, and use additional logic to determine whether the transaction can be committed
|
||||
// For example, prepareMSetNX will return error to prevent MSetNx transaction from committing if any related key already exists
|
||||
var prepareFuncMap = make(map[string]CmdFunc)
|
||||
|
||||
func registerPrepareFunc(cmdName string, fn CmdFunc) {
|
||||
prepareFuncMap[strings.ToLower(cmdName)] = fn
|
||||
}
|
||||
|
||||
// Transaction stores state and data for a try-commit-catch distributed transaction
|
||||
type Transaction struct {
|
||||
id string // transaction id
|
||||
cmdLine [][]byte // cmd cmdLine
|
||||
cluster *Cluster
|
||||
conn redis.Connection
|
||||
dbIndex int
|
||||
|
||||
writeKeys []string
|
||||
readKeys []string
|
||||
keysLocked bool
|
||||
undoLog []CmdLine
|
||||
|
||||
status int8
|
||||
mu *sync.Mutex
|
||||
}
|
||||
|
||||
const (
|
||||
maxLockTime = 3 * time.Second
|
||||
waitBeforeCleanTx = 2 * maxLockTime
|
||||
|
||||
createdStatus = 0
|
||||
preparedStatus = 1
|
||||
committedStatus = 2
|
||||
rolledBackStatus = 3
|
||||
)
|
||||
|
||||
func genTaskKey(txID string) string {
|
||||
return "tx:" + txID
|
||||
}
|
||||
|
||||
// NewTransaction creates a try-commit-catch distributed transaction
|
||||
func NewTransaction(cluster *Cluster, c redis.Connection, id string, cmdLine [][]byte) *Transaction {
|
||||
return &Transaction{
|
||||
id: id,
|
||||
cmdLine: cmdLine,
|
||||
cluster: cluster,
|
||||
conn: c,
|
||||
dbIndex: c.GetDBIndex(),
|
||||
status: createdStatus,
|
||||
mu: new(sync.Mutex),
|
||||
}
|
||||
}
|
||||
|
||||
// Reentrant
|
||||
// invoker should hold tx.mu
|
||||
func (tx *Transaction) lockKeys() {
|
||||
if !tx.keysLocked {
|
||||
tx.cluster.db.RWLocks(tx.dbIndex, tx.writeKeys, tx.readKeys)
|
||||
tx.keysLocked = true
|
||||
}
|
||||
}
|
||||
|
||||
func (tx *Transaction) unLockKeys() {
|
||||
if tx.keysLocked {
|
||||
tx.cluster.db.RWUnLocks(tx.dbIndex, tx.writeKeys, tx.readKeys)
|
||||
tx.keysLocked = false
|
||||
}
|
||||
}
|
||||
|
||||
// t should contain Keys and ID field
|
||||
func (tx *Transaction) prepare() error {
|
||||
tx.mu.Lock()
|
||||
defer tx.mu.Unlock()
|
||||
|
||||
tx.writeKeys, tx.readKeys = database.GetRelatedKeys(tx.cmdLine)
|
||||
// lock writeKeys
|
||||
tx.lockKeys()
|
||||
|
||||
for _, key := range tx.writeKeys {
|
||||
err := tx.cluster.ensureKey(key)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
for _, key := range tx.readKeys {
|
||||
err := tx.cluster.ensureKey(key)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// build undoLog
|
||||
tx.undoLog = tx.cluster.db.GetUndoLogs(tx.dbIndex, tx.cmdLine)
|
||||
tx.status = preparedStatus
|
||||
taskKey := genTaskKey(tx.id)
|
||||
timewheel.Delay(maxLockTime, taskKey, func() {
|
||||
if tx.status == preparedStatus { // rollback transaction uncommitted until expire
|
||||
logger.Info("abort transaction: " + tx.id)
|
||||
tx.mu.Lock()
|
||||
defer tx.mu.Unlock()
|
||||
_ = tx.rollbackWithLock()
|
||||
}
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (tx *Transaction) rollbackWithLock() error {
|
||||
curStatus := tx.status
|
||||
|
||||
if tx.status != curStatus { // ensure status not changed by other goroutine
|
||||
return fmt.Errorf("tx %s status changed", tx.id)
|
||||
}
|
||||
if tx.status == rolledBackStatus { // no need to rollback a rolled-back transaction
|
||||
return nil
|
||||
}
|
||||
tx.lockKeys()
|
||||
for _, cmdLine := range tx.undoLog {
|
||||
tx.cluster.db.ExecWithLock(tx.conn, cmdLine)
|
||||
}
|
||||
tx.unLockKeys()
|
||||
tx.status = rolledBackStatus
|
||||
return nil
|
||||
}
|
||||
|
||||
// cmdLine: Prepare id cmdName args...
|
||||
func execPrepare(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if len(cmdLine) < 3 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'prepare' command")
|
||||
}
|
||||
txID := string(cmdLine[1])
|
||||
cmdName := strings.ToLower(string(cmdLine[2]))
|
||||
tx := NewTransaction(cluster, c, txID, cmdLine[2:])
|
||||
cluster.transactionMu.Lock()
|
||||
cluster.transactions.Put(txID, tx)
|
||||
cluster.transactionMu.Unlock()
|
||||
err := tx.prepare()
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply(err.Error())
|
||||
}
|
||||
prepareFunc, ok := prepareFuncMap[cmdName]
|
||||
if ok {
|
||||
return prepareFunc(cluster, c, cmdLine[2:])
|
||||
}
|
||||
return &protocol.OkReply{}
|
||||
}
|
||||
|
||||
// execRollback rollbacks local transaction
|
||||
func execRollback(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if len(cmdLine) != 2 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'rollback' command")
|
||||
}
|
||||
txID := string(cmdLine[1])
|
||||
cluster.transactionMu.RLock()
|
||||
raw, ok := cluster.transactions.Get(txID)
|
||||
cluster.transactionMu.RUnlock()
|
||||
if !ok {
|
||||
return protocol.MakeIntReply(0)
|
||||
}
|
||||
tx, _ := raw.(*Transaction)
|
||||
|
||||
tx.mu.Lock()
|
||||
defer tx.mu.Unlock()
|
||||
err := tx.rollbackWithLock()
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply(err.Error())
|
||||
}
|
||||
// clean transaction
|
||||
timewheel.Delay(waitBeforeCleanTx, "", func() {
|
||||
cluster.transactionMu.Lock()
|
||||
cluster.transactions.Remove(tx.id)
|
||||
cluster.transactionMu.Unlock()
|
||||
})
|
||||
return protocol.MakeIntReply(1)
|
||||
}
|
||||
|
||||
// execCommit commits local transaction as a worker when receive execCommit command from coordinator
|
||||
func execCommit(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
if len(cmdLine) != 2 {
|
||||
return protocol.MakeErrReply("ERR wrong number of arguments for 'commit' command")
|
||||
}
|
||||
txID := string(cmdLine[1])
|
||||
cluster.transactionMu.RLock()
|
||||
raw, ok := cluster.transactions.Get(txID)
|
||||
cluster.transactionMu.RUnlock()
|
||||
if !ok {
|
||||
return protocol.MakeIntReply(0)
|
||||
}
|
||||
tx, _ := raw.(*Transaction)
|
||||
|
||||
tx.mu.Lock()
|
||||
defer tx.mu.Unlock()
|
||||
|
||||
result := cluster.db.ExecWithLock(c, tx.cmdLine)
|
||||
|
||||
if protocol.IsErrorReply(result) {
|
||||
// failed
|
||||
err2 := tx.rollbackWithLock()
|
||||
return protocol.MakeErrReply(fmt.Sprintf("err occurs when rollback: %v, origin err: %s", err2, result))
|
||||
}
|
||||
// after committed
|
||||
tx.unLockKeys()
|
||||
tx.status = committedStatus
|
||||
// clean finished transaction
|
||||
// do not clean immediately, in case rollback
|
||||
timewheel.Delay(waitBeforeCleanTx, "", func() {
|
||||
cluster.transactionMu.Lock()
|
||||
cluster.transactions.Remove(tx.id)
|
||||
cluster.transactionMu.Unlock()
|
||||
})
|
||||
return result
|
||||
}
|
||||
|
||||
// requestCommit commands all node to commit transaction as coordinator
|
||||
func requestCommit(cluster *Cluster, c redis.Connection, txID int64, groupMap map[string][]string) ([]redis.Reply, protocol.ErrorReply) {
|
||||
var errReply protocol.ErrorReply
|
||||
txIDStr := strconv.FormatInt(txID, 10)
|
||||
respList := make([]redis.Reply, 0, len(groupMap))
|
||||
for node := range groupMap {
|
||||
resp := cluster.relay(node, c, makeArgs("commit", txIDStr))
|
||||
if protocol.IsErrorReply(resp) {
|
||||
errReply = resp.(protocol.ErrorReply)
|
||||
break
|
||||
}
|
||||
respList = append(respList, resp)
|
||||
}
|
||||
if errReply != nil {
|
||||
requestRollback(cluster, c, txID, groupMap)
|
||||
return nil, errReply
|
||||
}
|
||||
return respList, nil
|
||||
}
|
||||
|
||||
// requestRollback requests all node rollback transaction as coordinator
|
||||
// groupMap: node -> keys
|
||||
func requestRollback(cluster *Cluster, c redis.Connection, txID int64, groupMap map[string][]string) {
|
||||
txIDStr := strconv.FormatInt(txID, 10)
|
||||
for node := range groupMap {
|
||||
cluster.relay(node, c, makeArgs("rollback", txIDStr))
|
||||
}
|
||||
}
|
||||
@@ -1,50 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"math/rand"
|
||||
"strconv"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestRollback(t *testing.T) {
|
||||
// rollback uncommitted transaction
|
||||
testNodeA := testCluster[0]
|
||||
conn := new(connection.FakeConn)
|
||||
FlushAll(testNodeA, conn, toArgs("FLUSHALL"))
|
||||
txID := rand.Int63()
|
||||
txIDStr := strconv.FormatInt(txID, 10)
|
||||
keys := []string{"a", "{a}1"}
|
||||
groupMap := map[string][]string{
|
||||
testNodeA.self: keys,
|
||||
}
|
||||
args := []string{txIDStr, "DEL"}
|
||||
args = append(args, keys...)
|
||||
testNodeA.db.Exec(conn, toArgs("SET", "a", "a"))
|
||||
ret := execPrepare(testNodeA, conn, makeArgs("Prepare", args...))
|
||||
asserts.AssertNotError(t, ret)
|
||||
requestRollback(testNodeA, conn, txID, groupMap)
|
||||
ret = testNodeA.db.Exec(conn, toArgs("GET", "a"))
|
||||
asserts.AssertBulkReply(t, ret, "a")
|
||||
|
||||
// rollback committed transaction
|
||||
FlushAll(testNodeA, conn, toArgs("FLUSHALL"))
|
||||
testNodeA.db.Exec(conn, toArgs("SET", "a", "a"))
|
||||
txID = rand.Int63()
|
||||
txIDStr = strconv.FormatInt(txID, 10)
|
||||
args = []string{txIDStr, "DEL"}
|
||||
args = append(args, keys...)
|
||||
ret = execPrepare(testNodeA, conn, makeArgs("Prepare", args...))
|
||||
asserts.AssertNotError(t, ret)
|
||||
_, err := requestCommit(testNodeA, conn, txID, groupMap)
|
||||
if err != nil {
|
||||
t.Errorf("del failed %v", err)
|
||||
return
|
||||
}
|
||||
ret = testNodeA.db.Exec(conn, toArgs("GET", "a")) // call db.Exec to skip key router
|
||||
asserts.AssertNullBulk(t, ret)
|
||||
requestRollback(testNodeA, conn, txID, groupMap)
|
||||
ret = testNodeA.db.Exec(conn, toArgs("GET", "a"))
|
||||
asserts.AssertBulkReply(t, ret, "a")
|
||||
}
|
||||
189
cluster/topo.go
189
cluster/topo.go
@@ -1,189 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/hdt3213/godis/database"
|
||||
"github.com/hdt3213/godis/lib/logger"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"strconv"
|
||||
"time"
|
||||
)
|
||||
|
||||
func (cluster *Cluster) startAsSeed(listenAddr string) protocol.ErrorReply {
|
||||
err := cluster.topology.StartAsSeed(listenAddr)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for i := 0; i < slotCount; i++ {
|
||||
cluster.initSlot(uint32(i), slotStateHost)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Join send `gcluster join` to node in cluster to join
|
||||
func (cluster *Cluster) Join(seed string) protocol.ErrorReply {
|
||||
err := cluster.topology.Join(seed)
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
/* STEP3: asynchronous migrating slots */
|
||||
go func() {
|
||||
time.Sleep(time.Second) // let the cluster started
|
||||
cluster.reBalance()
|
||||
}()
|
||||
return nil
|
||||
}
|
||||
|
||||
var errConfigFileNotExist = protocol.MakeErrReply("cluster config file not exist")
|
||||
|
||||
// LoadConfig try to load cluster-config-file and re-join the cluster
|
||||
func (cluster *Cluster) LoadConfig() protocol.ErrorReply {
|
||||
err := cluster.topology.LoadConfigFile()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
selfNodeId := cluster.topology.GetSelfNodeID()
|
||||
selfNode := cluster.topology.GetNode(selfNodeId)
|
||||
if selfNode == nil {
|
||||
return protocol.MakeErrReply("ERR self node info not found")
|
||||
}
|
||||
for _, slot := range selfNode.Slots {
|
||||
cluster.initSlot(slot.ID, slotStateHost)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (cluster *Cluster) reBalance() {
|
||||
nodes := cluster.topology.GetNodes()
|
||||
var slotIDs []uint32
|
||||
var slots []*Slot
|
||||
reqDonateCmdLine := utils.ToCmdLine("gcluster", "request-donate", cluster.self)
|
||||
for _, node := range nodes {
|
||||
if node.ID == cluster.self {
|
||||
continue
|
||||
}
|
||||
node := node
|
||||
peerCli, err := cluster.clientFactory.GetPeerClient(node.Addr)
|
||||
if err != nil {
|
||||
logger.Errorf("get client of %s failed: %v", node.Addr, err)
|
||||
continue
|
||||
}
|
||||
resp := peerCli.Send(reqDonateCmdLine)
|
||||
payload, ok := resp.(*protocol.MultiBulkReply)
|
||||
if !ok {
|
||||
logger.Errorf("request donate to %s failed: %v", node.Addr, err)
|
||||
continue
|
||||
}
|
||||
for _, bin := range payload.Args {
|
||||
slotID64, err := strconv.ParseUint(string(bin), 10, 64)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
slotID := uint32(slotID64)
|
||||
slotIDs = append(slotIDs, slotID)
|
||||
slots = append(slots, &Slot{
|
||||
ID: slotID,
|
||||
NodeID: node.ID,
|
||||
})
|
||||
// Raft cannot guarantee the simultaneity and order of submissions to the source and destination nodes
|
||||
// In some cases the source node thinks the slot belongs to the destination node, and the destination node thinks the slot belongs to the source node
|
||||
// To avoid it, the source node and the destination node must reach a consensus before propose to raft
|
||||
cluster.setLocalSlotImporting(slotID, node.ID)
|
||||
}
|
||||
}
|
||||
if len(slots) == 0 {
|
||||
return
|
||||
}
|
||||
logger.Infof("received %d donated slots", len(slots))
|
||||
|
||||
// change route
|
||||
err := cluster.topology.SetSlot(slotIDs, cluster.self)
|
||||
if err != nil {
|
||||
logger.Errorf("set slot route failed: %v", err)
|
||||
return
|
||||
}
|
||||
slotChan := make(chan *Slot, len(slots))
|
||||
for _, slot := range slots {
|
||||
slotChan <- slot
|
||||
}
|
||||
close(slotChan)
|
||||
for i := 0; i < 4; i++ {
|
||||
i := i
|
||||
go func() {
|
||||
for slot := range slotChan {
|
||||
logger.Info("start import slot ", slot.ID)
|
||||
err := cluster.importSlot(slot)
|
||||
if err != nil {
|
||||
logger.Error(fmt.Sprintf("import slot %d error: %v", slot.ID, err))
|
||||
// delete all imported keys in slot
|
||||
cluster.cleanDroppedSlot(slot.ID)
|
||||
// todo: recover route
|
||||
return
|
||||
}
|
||||
logger.Infof("finish import slot: %d, about %d slots remains", slot.ID, len(slotChan))
|
||||
}
|
||||
logger.Infof("import worker %d exited", i)
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
// importSlot do migrate slot into current node
|
||||
// the pseudo `slot` parameter is used to store slotID and former host node
|
||||
func (cluster *Cluster) importSlot(slot *Slot) error {
|
||||
node := cluster.topology.GetNode(slot.NodeID)
|
||||
|
||||
/* get migrate stream */
|
||||
migrateCmdLine := utils.ToCmdLine(
|
||||
"gcluster", "migrate", strconv.Itoa(int(slot.ID)))
|
||||
migrateStream, err := cluster.clientFactory.NewStream(node.Addr, migrateCmdLine)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer migrateStream.Close()
|
||||
|
||||
fakeConn := connection.NewFakeConn()
|
||||
slotLoop:
|
||||
for proto := range migrateStream.Stream() {
|
||||
if proto.Err != nil {
|
||||
return fmt.Errorf("set slot %d error: %v", slot.ID, err)
|
||||
}
|
||||
switch reply := proto.Data.(type) {
|
||||
case *protocol.MultiBulkReply:
|
||||
// todo: handle exec error
|
||||
keys, _ := database.GetRelatedKeys(reply.Args)
|
||||
// assert len(keys) == 1
|
||||
key := keys[0]
|
||||
// key may be imported by Cluster.ensureKey or by former failed migrating try
|
||||
if !cluster.isImportedKey(key) {
|
||||
cluster.setImportedKey(key)
|
||||
_ = cluster.db.Exec(fakeConn, reply.Args)
|
||||
}
|
||||
case *protocol.StatusReply:
|
||||
if protocol.IsOKReply(reply) {
|
||||
break slotLoop
|
||||
} else {
|
||||
// todo: return slot to former host node
|
||||
msg := fmt.Sprintf("migrate slot %d error: %s", slot.ID, reply.Status)
|
||||
logger.Errorf(msg)
|
||||
return protocol.MakeErrReply(msg)
|
||||
}
|
||||
case protocol.ErrorReply:
|
||||
// todo: return slot to former host node
|
||||
msg := fmt.Sprintf("migrate slot %d error: %s", slot.ID, reply.Error())
|
||||
logger.Errorf(msg)
|
||||
return protocol.MakeErrReply(msg)
|
||||
}
|
||||
}
|
||||
cluster.finishSlotImport(slot.ID)
|
||||
|
||||
// finish migration mode
|
||||
peerCli, err := cluster.clientFactory.GetPeerClient(node.Addr)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer cluster.clientFactory.ReturnPeerClient(node.Addr, peerCli)
|
||||
peerCli.Send(utils.ToCmdLine("gcluster", "migrate-done", strconv.Itoa(int(slot.ID))))
|
||||
return nil
|
||||
}
|
||||
@@ -1,146 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/hdt3213/godis/aof"
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/logger"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"strconv"
|
||||
"strings"
|
||||
)
|
||||
|
||||
func init() {
|
||||
registerCmd("gcluster", execGCluster)
|
||||
}
|
||||
|
||||
func execGCluster(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
if len(args) < 2 {
|
||||
return protocol.MakeArgNumErrReply("gcluster")
|
||||
}
|
||||
subCmd := strings.ToLower(string(args[1]))
|
||||
switch subCmd {
|
||||
case "set-slot":
|
||||
// Command line: gcluster set-slot <slotID> <targetNodeID>
|
||||
// Other node request current node to migrate a slot to it.
|
||||
// Current node will set the slot as migrating state.
|
||||
// After this function return, all requests of target slot will be routed to target node
|
||||
return execGClusterSetSlot(cluster, c, args[2:])
|
||||
case "migrate":
|
||||
// Command line: gcluster migrate <slotId>
|
||||
// Current node will dump the given slot to the node sending this request
|
||||
// The given slot must in migrating state
|
||||
return execGClusterMigrate(cluster, c, args[2:])
|
||||
case "migrate-done":
|
||||
// command line: gcluster migrate-done <slotId>
|
||||
// The new node hosting given slot tells current node that migration has finished, remains data can be deleted
|
||||
return execGClusterMigrateDone(cluster, c, args[2:])
|
||||
case "request-donate":
|
||||
// command line: gcluster donate <nodeID>
|
||||
// picks some slots and gives them to the calling node for load balance
|
||||
return execGClusterDonateSlot(cluster, c, args[2:])
|
||||
}
|
||||
return protocol.MakeErrReply(" ERR unknown gcluster sub command '" + subCmd + "'")
|
||||
}
|
||||
|
||||
// execGClusterSetSlot set a current node hosted slot as migrating
|
||||
// args is [slotID, newNodeId]
|
||||
func execGClusterSetSlot(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
if len(args) != 2 {
|
||||
return protocol.MakeArgNumErrReply("gcluster")
|
||||
}
|
||||
slotId0, err := strconv.Atoi(string(args[0]))
|
||||
if err != nil || slotId0 >= slotCount {
|
||||
return protocol.MakeErrReply("ERR value is not a valid slot id")
|
||||
}
|
||||
slotId := uint32(slotId0)
|
||||
targetNodeID := string(args[1])
|
||||
targetNode := cluster.topology.GetNode(targetNodeID)
|
||||
if targetNode == nil {
|
||||
return protocol.MakeErrReply("ERR node not found")
|
||||
}
|
||||
cluster.setSlotMovingOut(slotId, targetNodeID)
|
||||
logger.Info(fmt.Sprintf("set slot %d to node %s", slotId, targetNodeID))
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
|
||||
// execGClusterDonateSlot picks some slots and gives them to the calling node for load balance
|
||||
// args is [callingNodeId]
|
||||
func execGClusterDonateSlot(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
targetNodeID := string(args[0])
|
||||
nodes := cluster.topology.GetNodes() // including the new node
|
||||
avgSlot := slotCount / len(nodes)
|
||||
cluster.slotMu.Lock()
|
||||
defer cluster.slotMu.Unlock()
|
||||
limit := len(cluster.slots) - avgSlot
|
||||
if limit <= 0 {
|
||||
return protocol.MakeEmptyMultiBulkReply()
|
||||
}
|
||||
result := make([][]byte, 0, limit)
|
||||
// use the randomness of the for-each-in-map to randomly select slots
|
||||
for slotID, slot := range cluster.slots {
|
||||
if slot.state == slotStateHost {
|
||||
slot.state = slotStateMovingOut
|
||||
slot.newNodeID = targetNodeID
|
||||
slotIDBin := []byte(strconv.FormatUint(uint64(slotID), 10))
|
||||
result = append(result, slotIDBin)
|
||||
if len(result) == limit {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
return protocol.MakeMultiBulkReply(result)
|
||||
}
|
||||
|
||||
// execGClusterMigrate Command line: gcluster migrate slotId
|
||||
// Current node will dump data in the given slot to the node sending this request
|
||||
// The given slot must in migrating state
|
||||
func execGClusterMigrate(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
slotId0, err := strconv.Atoi(string(args[0]))
|
||||
if err != nil || slotId0 >= slotCount {
|
||||
return protocol.MakeErrReply("ERR value is not a valid slot id")
|
||||
}
|
||||
slotId := uint32(slotId0)
|
||||
slot := cluster.getHostSlot(slotId)
|
||||
if slot == nil || slot.state != slotStateMovingOut {
|
||||
return protocol.MakeErrReply("ERR only dump migrating slot")
|
||||
}
|
||||
// migrating slot is immutable
|
||||
logger.Info("start dump slot", slotId)
|
||||
slot.keys.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
|
||||
})
|
||||
logger.Info("finish dump slot ", slotId)
|
||||
// send a ok reply to tell requesting node dump finished
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
|
||||
// execGClusterMigrateDone command line: gcluster migrate-done <slotId>
|
||||
func execGClusterMigrateDone(cluster *Cluster, c redis.Connection, args [][]byte) redis.Reply {
|
||||
slotId0, err := strconv.Atoi(string(args[0]))
|
||||
if err != nil || slotId0 >= slotCount {
|
||||
return protocol.MakeErrReply("ERR value is not a valid slot id")
|
||||
}
|
||||
slotId := uint32(slotId0)
|
||||
slot := cluster.getHostSlot(slotId)
|
||||
if slot == nil || slot.state != slotStateMovingOut {
|
||||
return protocol.MakeErrReply("ERR slot is not moving out")
|
||||
}
|
||||
cluster.cleanDroppedSlot(slotId)
|
||||
cluster.slotMu.Lock()
|
||||
delete(cluster.slots, slotId)
|
||||
cluster.slotMu.Unlock()
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
@@ -1,58 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"hash/crc32"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
// Slot represents a hash slot, used in cluster internal messages
|
||||
type Slot struct {
|
||||
// ID is uint between 0 and 16383
|
||||
ID uint32
|
||||
// NodeID is id of the hosting node
|
||||
// If the slot is migrating, NodeID is the id of the node importing this slot (target node)
|
||||
NodeID string
|
||||
// Flags stores more information of slot
|
||||
Flags uint32
|
||||
}
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
||||
// Node represents a node and its slots, used in cluster internal messages
|
||||
type Node struct {
|
||||
ID string
|
||||
Addr string
|
||||
Slots []*Slot // ascending order by slot id
|
||||
Flags uint32
|
||||
lastHeard time.Time
|
||||
}
|
||||
|
||||
type topology interface {
|
||||
GetSelfNodeID() string
|
||||
GetNodes() []*Node // return a copy
|
||||
GetNode(nodeID string) *Node
|
||||
GetSlots() []*Slot
|
||||
StartAsSeed(addr string) protocol.ErrorReply
|
||||
SetSlot(slotIDs []uint32, newNodeID string) protocol.ErrorReply
|
||||
LoadConfigFile() protocol.ErrorReply
|
||||
Join(seed string) protocol.ErrorReply
|
||||
Close() error
|
||||
}
|
||||
@@ -1,152 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/config"
|
||||
database2 "github.com/hdt3213/godis/database"
|
||||
"github.com/hdt3213/godis/datastruct/dict"
|
||||
"github.com/hdt3213/godis/lib/idgenerator"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"path"
|
||||
"strconv"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func makeTestRaft(addresses []string, timeoutFlags []bool, persistFilenames []string) ([]*Cluster, error) {
|
||||
nodes := make([]*Cluster, len(addresses))
|
||||
factory := &testClientFactory{
|
||||
nodes: nodes,
|
||||
timeoutFlags: timeoutFlags,
|
||||
}
|
||||
for i, addr := range addresses {
|
||||
addr := addr
|
||||
nodes[i] = &Cluster{
|
||||
self: addr,
|
||||
addr: addr,
|
||||
db: database2.NewStandaloneServer(),
|
||||
transactions: dict.MakeSimple(),
|
||||
idGenerator: idgenerator.MakeGenerator(config.Properties.Self),
|
||||
clientFactory: factory,
|
||||
slots: make(map[uint32]*hostSlot),
|
||||
}
|
||||
topologyPersistFile := persistFilenames[i]
|
||||
nodes[i].topology = newRaft(nodes[i], topologyPersistFile)
|
||||
}
|
||||
|
||||
err := nodes[0].startAsSeed(addresses[0])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = nodes[1].Join(addresses[0])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = nodes[2].Join(addresses[0])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return nodes, nil
|
||||
}
|
||||
|
||||
func TestRaftStart(t *testing.T) {
|
||||
addresses := []string{"127.0.0.1:6399", "127.0.0.1:7379", "127.0.0.1:7369"}
|
||||
timeoutFlags := []bool{false, false, false}
|
||||
persistFilenames := []string{"", "", ""}
|
||||
nodes, err := makeTestRaft(addresses, timeoutFlags, persistFilenames)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
}
|
||||
if nodes[0].asRaft().state != leader {
|
||||
t.Error("expect leader")
|
||||
return
|
||||
}
|
||||
if nodes[1].asRaft().state != follower {
|
||||
t.Error("expect follower")
|
||||
return
|
||||
}
|
||||
if nodes[2].asRaft().state != follower {
|
||||
t.Error("expect follower")
|
||||
return
|
||||
}
|
||||
size := 100
|
||||
conn := connection.NewFakeConn()
|
||||
for i := 0; i < size; i++ {
|
||||
str := strconv.Itoa(i)
|
||||
result := nodes[0].Exec(conn, utils.ToCmdLine("SET", str, str))
|
||||
asserts.AssertNotError(t, result)
|
||||
}
|
||||
for i := 0; i < size; i++ {
|
||||
str := strconv.Itoa(i)
|
||||
result := nodes[0].Exec(conn, utils.ToCmdLine("Get", str))
|
||||
asserts.AssertBulkReply(t, result, str)
|
||||
}
|
||||
for _, node := range nodes {
|
||||
_ = node.asRaft().Close()
|
||||
}
|
||||
}
|
||||
|
||||
func TestRaftElection(t *testing.T) {
|
||||
addresses := []string{"127.0.0.1:6399", "127.0.0.1:7379", "127.0.0.1:7369"}
|
||||
timeoutFlags := []bool{false, false, false}
|
||||
persistFilenames := []string{"", "", ""}
|
||||
nodes, err := makeTestRaft(addresses, timeoutFlags, persistFilenames)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
}
|
||||
nodes[0].asRaft().Close()
|
||||
time.Sleep(3 * electionTimeoutMaxMs * time.Millisecond) // wait for leader timeout
|
||||
//<-make(chan struct{}) // wait for leader timeout
|
||||
for i := 0; i < 10; i++ {
|
||||
leaderCount := 0
|
||||
for _, node := range nodes {
|
||||
if node.asRaft().closed {
|
||||
continue
|
||||
}
|
||||
switch node.asRaft().state {
|
||||
case leader:
|
||||
leaderCount++
|
||||
}
|
||||
}
|
||||
if leaderCount == 1 {
|
||||
break
|
||||
} else if leaderCount > 1 {
|
||||
t.Errorf("get %d leaders, split brain", leaderCount)
|
||||
break
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
}
|
||||
|
||||
func TestRaftPersist(t *testing.T) {
|
||||
addresses := []string{"127.0.0.1:6399", "127.0.0.1:7379", "127.0.0.1:7369"}
|
||||
timeoutFlags := []bool{false, false, false}
|
||||
persistFilenames := []string{
|
||||
path.Join(config.Properties.Dir, "test6399.conf"),
|
||||
path.Join(config.Properties.Dir, "test7379.conf"),
|
||||
path.Join(config.Properties.Dir, "test7369.conf"),
|
||||
}
|
||||
nodes, err := makeTestRaft(addresses, timeoutFlags, persistFilenames)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
}
|
||||
node1 := nodes[0].asRaft()
|
||||
err = node1.persist()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
}
|
||||
for _, node := range nodes {
|
||||
_ = node.asRaft().Close()
|
||||
}
|
||||
|
||||
err = node1.LoadConfigFile()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
}
|
||||
}
|
||||
@@ -1,99 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/datastruct/set"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
)
|
||||
|
||||
func (cluster *Cluster) isImportedKey(key string) bool {
|
||||
slotId := getSlot(key)
|
||||
cluster.slotMu.RLock()
|
||||
slot := cluster.slots[slotId]
|
||||
cluster.slotMu.RUnlock()
|
||||
return slot.importedKeys.Has(key)
|
||||
}
|
||||
|
||||
func (cluster *Cluster) setImportedKey(key string) {
|
||||
slotId := getSlot(key)
|
||||
cluster.slotMu.Lock()
|
||||
slot := cluster.slots[slotId]
|
||||
cluster.slotMu.Unlock()
|
||||
slot.importedKeys.Add(key)
|
||||
}
|
||||
|
||||
// initSlot init a slot when start as seed or import slot from other node
|
||||
func (cluster *Cluster) initSlot(slotId uint32, state uint32) {
|
||||
cluster.slotMu.Lock()
|
||||
defer cluster.slotMu.Unlock()
|
||||
cluster.slots[slotId] = &hostSlot{
|
||||
importedKeys: set.Make(),
|
||||
keys: set.Make(),
|
||||
state: state,
|
||||
}
|
||||
}
|
||||
|
||||
func (cluster *Cluster) getHostSlot(slotId uint32) *hostSlot {
|
||||
cluster.slotMu.RLock()
|
||||
defer cluster.slotMu.RUnlock()
|
||||
return cluster.slots[slotId]
|
||||
}
|
||||
|
||||
func (cluster *Cluster) finishSlotImport(slotID uint32) {
|
||||
cluster.slotMu.Lock()
|
||||
defer cluster.slotMu.Unlock()
|
||||
slot := cluster.slots[slotID]
|
||||
slot.state = slotStateHost
|
||||
slot.importedKeys = nil
|
||||
slot.oldNodeID = ""
|
||||
}
|
||||
|
||||
func (cluster *Cluster) setLocalSlotImporting(slotID uint32, oldNodeID string) {
|
||||
cluster.slotMu.Lock()
|
||||
defer cluster.slotMu.Unlock()
|
||||
slot := cluster.slots[slotID]
|
||||
if slot == nil {
|
||||
slot = &hostSlot{
|
||||
importedKeys: set.Make(),
|
||||
keys: set.Make(),
|
||||
}
|
||||
cluster.slots[slotID] = slot
|
||||
}
|
||||
slot.state = slotStateImporting
|
||||
slot.oldNodeID = oldNodeID
|
||||
}
|
||||
|
||||
func (cluster *Cluster) setSlotMovingOut(slotID uint32, newNodeID string) {
|
||||
cluster.slotMu.Lock()
|
||||
defer cluster.slotMu.Unlock()
|
||||
slot := cluster.slots[slotID]
|
||||
if slot == nil {
|
||||
slot = &hostSlot{
|
||||
importedKeys: set.Make(),
|
||||
keys: set.Make(),
|
||||
}
|
||||
cluster.slots[slotID] = slot
|
||||
}
|
||||
slot.state = slotStateMovingOut
|
||||
slot.newNodeID = newNodeID
|
||||
}
|
||||
|
||||
// cleanDroppedSlot deletes keys when slot has moved out or failed to import
|
||||
func (cluster *Cluster) cleanDroppedSlot(slotID uint32) {
|
||||
cluster.slotMu.RLock()
|
||||
if cluster.slots[slotID] == nil {
|
||||
cluster.slotMu.RUnlock()
|
||||
return
|
||||
}
|
||||
keys := cluster.slots[slotID].importedKeys
|
||||
cluster.slotMu.RUnlock()
|
||||
c := connection.NewFakeConn()
|
||||
go func() {
|
||||
if keys != nil {
|
||||
keys.ForEach(func(key string) bool {
|
||||
cluster.db.Exec(c, utils.ToCmdLine("DEL", key))
|
||||
return true
|
||||
})
|
||||
}
|
||||
}()
|
||||
}
|
||||
@@ -1,75 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
)
|
||||
|
||||
func ping(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
return cluster.db.Exec(c, cmdLine)
|
||||
}
|
||||
|
||||
func info(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
return cluster.db.Exec(c, cmdLine)
|
||||
}
|
||||
|
||||
func randomkey(cluster *Cluster, c redis.Connection, cmdLine CmdLine) redis.Reply {
|
||||
return cluster.db.Exec(c, cmdLine)
|
||||
}
|
||||
|
||||
/*----- utils -------*/
|
||||
|
||||
func makeArgs(cmd string, args ...string) [][]byte {
|
||||
result := make([][]byte, len(args)+1)
|
||||
result[0] = []byte(cmd)
|
||||
for i, arg := range args {
|
||||
result[i+1] = []byte(arg)
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
// return node -> writeKeys
|
||||
func (cluster *Cluster) groupBy(keys []string) map[string][]string {
|
||||
result := make(map[string][]string)
|
||||
for _, key := range keys {
|
||||
peer := cluster.pickNodeAddrByKey(key)
|
||||
group, ok := result[peer]
|
||||
if !ok {
|
||||
group = make([]string, 0)
|
||||
}
|
||||
group = append(group, key)
|
||||
result[peer] = group
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
// pickNode returns the node id hosting the given slot.
|
||||
// If the slot is migrating, return the node which is importing the slot
|
||||
func (cluster *Cluster) pickNode(slotID uint32) *Node {
|
||||
// check cluster.slot to avoid errors caused by inconsistent status on follower nodes during raft commits
|
||||
// see cluster.reBalance()
|
||||
hSlot := cluster.getHostSlot(slotID)
|
||||
if hSlot != nil {
|
||||
switch hSlot.state {
|
||||
case slotStateMovingOut:
|
||||
return cluster.topology.GetNode(hSlot.newNodeID)
|
||||
case slotStateImporting, slotStateHost:
|
||||
return cluster.topology.GetNode(cluster.self)
|
||||
}
|
||||
}
|
||||
|
||||
slot := cluster.topology.GetSlots()[int(slotID)]
|
||||
node := cluster.topology.GetNode(slot.NodeID)
|
||||
return node
|
||||
}
|
||||
|
||||
func (cluster *Cluster) pickNodeAddrByKey(key string) string {
|
||||
slotId := getSlot(key)
|
||||
return cluster.pickNode(slotId).Addr
|
||||
}
|
||||
|
||||
func modifyCmd(cmdLine CmdLine, newCmd string) CmdLine {
|
||||
var cmdLine2 CmdLine
|
||||
cmdLine2 = append(cmdLine2, cmdLine...)
|
||||
cmdLine2[0] = []byte(newCmd)
|
||||
return cmdLine2
|
||||
}
|
||||
@@ -1,159 +0,0 @@
|
||||
package cluster
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"github.com/hdt3213/godis/config"
|
||||
database2 "github.com/hdt3213/godis/database"
|
||||
"github.com/hdt3213/godis/datastruct/dict"
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/lib/idgenerator"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/parser"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"math/rand"
|
||||
"sync"
|
||||
)
|
||||
|
||||
type testClientFactory struct {
|
||||
nodes []*Cluster
|
||||
timeoutFlags []bool
|
||||
}
|
||||
|
||||
type testClient struct {
|
||||
targetNode *Cluster
|
||||
timeoutFlag *bool
|
||||
conn redis.Connection
|
||||
}
|
||||
|
||||
func (cli *testClient) Send(cmdLine [][]byte) redis.Reply {
|
||||
if *cli.timeoutFlag {
|
||||
return protocol.MakeErrReply("ERR timeout")
|
||||
}
|
||||
return cli.targetNode.Exec(cli.conn, cmdLine)
|
||||
}
|
||||
|
||||
func (factory *testClientFactory) GetPeerClient(peerAddr string) (peerClient, error) {
|
||||
for i, n := range factory.nodes {
|
||||
if n.self == peerAddr {
|
||||
cli := &testClient{
|
||||
targetNode: n,
|
||||
timeoutFlag: &factory.timeoutFlags[i],
|
||||
conn: connection.NewFakeConn(),
|
||||
}
|
||||
if config.Properties.RequirePass != "" {
|
||||
cli.Send(utils.ToCmdLine("AUTH", config.Properties.RequirePass))
|
||||
}
|
||||
return cli, nil
|
||||
}
|
||||
}
|
||||
return nil, errors.New("peer not found")
|
||||
}
|
||||
|
||||
type mockStream struct {
|
||||
targetNode *Cluster
|
||||
ch <-chan *parser.Payload
|
||||
}
|
||||
|
||||
func (s *mockStream) Stream() <-chan *parser.Payload {
|
||||
return s.ch
|
||||
}
|
||||
|
||||
func (s *mockStream) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (factory *testClientFactory) NewStream(peerAddr string, cmdLine CmdLine) (peerStream, error) {
|
||||
for _, n := range factory.nodes {
|
||||
if n.self == peerAddr {
|
||||
conn := connection.NewFakeConn()
|
||||
if config.Properties.RequirePass != "" {
|
||||
n.Exec(conn, utils.ToCmdLine("AUTH", config.Properties.RequirePass))
|
||||
}
|
||||
result := n.Exec(conn, cmdLine)
|
||||
conn.Write(result.ToBytes())
|
||||
ch := parser.ParseStream(conn)
|
||||
return &mockStream{
|
||||
targetNode: n,
|
||||
ch: ch,
|
||||
}, nil
|
||||
}
|
||||
}
|
||||
return nil, errors.New("node not found")
|
||||
}
|
||||
|
||||
func (factory *testClientFactory) ReturnPeerClient(peer string, peerClient peerClient) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (factory *testClientFactory) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// mockClusterNodes creates a fake cluster for test
|
||||
// timeoutFlags should have the same length as addresses, set timeoutFlags[i] == true could simulate addresses[i] timeout
|
||||
func mockClusterNodes(addresses []string, timeoutFlags []bool) []*Cluster {
|
||||
nodes := make([]*Cluster, len(addresses))
|
||||
// build fixedTopology
|
||||
slots := make([]*Slot, slotCount)
|
||||
nodeMap := make(map[string]*Node)
|
||||
for _, addr := range addresses {
|
||||
nodeMap[addr] = &Node{
|
||||
ID: addr,
|
||||
Addr: addr,
|
||||
Slots: nil,
|
||||
}
|
||||
}
|
||||
for i := range slots {
|
||||
addr := addresses[i%len(addresses)]
|
||||
slots[i] = &Slot{
|
||||
ID: uint32(i),
|
||||
NodeID: addr,
|
||||
Flags: 0,
|
||||
}
|
||||
nodeMap[addr].Slots = append(nodeMap[addr].Slots, slots[i])
|
||||
}
|
||||
factory := &testClientFactory{
|
||||
nodes: nodes,
|
||||
timeoutFlags: timeoutFlags,
|
||||
}
|
||||
for i, addr := range addresses {
|
||||
topo := &fixedTopology{
|
||||
mu: sync.RWMutex{},
|
||||
nodeMap: nodeMap,
|
||||
slots: slots,
|
||||
selfNodeID: addr,
|
||||
}
|
||||
nodes[i] = &Cluster{
|
||||
self: addr,
|
||||
db: database2.NewStandaloneServer(),
|
||||
transactions: dict.MakeSimple(),
|
||||
idGenerator: idgenerator.MakeGenerator(config.Properties.Self),
|
||||
topology: topo,
|
||||
clientFactory: factory,
|
||||
}
|
||||
}
|
||||
return nodes
|
||||
}
|
||||
|
||||
var addresses = []string{"127.0.0.1:6399", "127.0.0.1:7379"}
|
||||
var timeoutFlags = []bool{false, false}
|
||||
var testCluster = mockClusterNodes(addresses, timeoutFlags)
|
||||
|
||||
func toArgs(cmd ...string) [][]byte {
|
||||
args := make([][]byte, len(cmd))
|
||||
for i, s := range cmd {
|
||||
args[i] = []byte(s)
|
||||
}
|
||||
return args
|
||||
}
|
||||
|
||||
var letters = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789")
|
||||
|
||||
func RandString(n int) string {
|
||||
b := make([]rune, n)
|
||||
for i := range b {
|
||||
b[i] = letters[rand.Intn(len(letters))]
|
||||
}
|
||||
return string(b)
|
||||
}
|
||||
@@ -40,16 +40,12 @@ type ServerProperties struct {
|
||||
SlaveAnnouncePort int `cfg:"slave-announce-port"`
|
||||
SlaveAnnounceIP string `cfg:"slave-announce-ip"`
|
||||
ReplTimeout int `cfg:"repl-timeout"`
|
||||
|
||||
ClusterEnable bool `cfg:"cluster-enable"`
|
||||
ClusterAsSeed bool `cfg:"cluster-as-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"`
|
||||
|
||||
RaftListenAddr string `cfg:"raft-listen-address"`
|
||||
RaftAdvertiseAddr string `cfg:"raft-advertise-address"`
|
||||
// config file path
|
||||
CfPath string `cfg:"cf,omitempty"`
|
||||
}
|
||||
@@ -59,7 +55,10 @@ type ServerInfo struct {
|
||||
}
|
||||
|
||||
func (p *ServerProperties) AnnounceAddress() string {
|
||||
return p.AnnounceHost + ":" + strconv.Itoa(p.Port)
|
||||
if p.AnnounceHost != "" {
|
||||
return p.AnnounceHost + ":" + strconv.Itoa(p.Port)
|
||||
}
|
||||
return p.Bind + ":" + strconv.Itoa(p.Port)
|
||||
}
|
||||
|
||||
// Properties holds global config properties
|
||||
|
||||
@@ -164,7 +164,7 @@ func GenGodisInfoString(section string, db *Server) []byte {
|
||||
|
||||
// getGodisRunningMode return godis running mode
|
||||
func getGodisRunningMode() string {
|
||||
if config.Properties.ClusterEnabled == "yes" {
|
||||
if config.Properties.ClusterEnable {
|
||||
return config.ClusterMode
|
||||
} else {
|
||||
return config.StandaloneMode
|
||||
|
||||
@@ -40,6 +40,18 @@ func computeCapacity(param int) (size int) {
|
||||
|
||||
// MakeConcurrent creates ConcurrentDict with the given shard count
|
||||
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)
|
||||
table := make([]*shard, shardCount)
|
||||
for i := 0; i < shardCount; i++ {
|
||||
@@ -66,10 +78,14 @@ func fnv32(key string) uint32 {
|
||||
return hash
|
||||
}
|
||||
|
||||
func (dict *ConcurrentDict) spread(hashCode uint32) uint32 {
|
||||
func (dict *ConcurrentDict) spread(key string) uint32 {
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
if len(dict.table) == 1 {
|
||||
return 0
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
tableSize := uint32(len(dict.table))
|
||||
return (tableSize - 1) & hashCode
|
||||
}
|
||||
@@ -86,8 +102,7 @@ func (dict *ConcurrentDict) Get(key string) (val interface{}, exists bool) {
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
@@ -99,8 +114,7 @@ func (dict *ConcurrentDict) GetWithLock(key string) (val interface{}, exists boo
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
val, exists = s.m[key]
|
||||
return
|
||||
@@ -119,8 +133,7 @@ func (dict *ConcurrentDict) Put(key string, val interface{}) (result int) {
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
@@ -138,8 +151,7 @@ func (dict *ConcurrentDict) PutWithLock(key string, val interface{}) (result int
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
|
||||
if _, ok := s.m[key]; ok {
|
||||
@@ -156,8 +168,7 @@ func (dict *ConcurrentDict) PutIfAbsent(key string, val interface{}) (result int
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
@@ -174,8 +185,7 @@ func (dict *ConcurrentDict) PutIfAbsentWithLock(key string, val interface{}) (re
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
|
||||
if _, ok := s.m[key]; ok {
|
||||
@@ -191,8 +201,7 @@ func (dict *ConcurrentDict) PutIfExists(key string, val interface{}) (result int
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
@@ -208,8 +217,7 @@ func (dict *ConcurrentDict) PutIfExistsWithLock(key string, val interface{}) (re
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
|
||||
if _, ok := s.m[key]; ok {
|
||||
@@ -224,8 +232,7 @@ func (dict *ConcurrentDict) Remove(key string) (val interface{}, result int) {
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
s.mutex.Lock()
|
||||
defer s.mutex.Unlock()
|
||||
@@ -242,8 +249,7 @@ func (dict *ConcurrentDict) RemoveWithLock(key string) (val interface{}, result
|
||||
if dict == nil {
|
||||
panic("dict is nil")
|
||||
}
|
||||
hashCode := fnv32(key)
|
||||
index := dict.spread(hashCode)
|
||||
index := dict.spread(key)
|
||||
s := dict.getShard(index)
|
||||
|
||||
if val, ok := s.m[key]; ok {
|
||||
@@ -381,7 +387,7 @@ func (dict *ConcurrentDict) Clear() {
|
||||
func (dict *ConcurrentDict) toLockIndices(keys []string, reverse bool) []uint32 {
|
||||
indexMap := make(map[uint32]struct{})
|
||||
for _, key := range keys {
|
||||
index := dict.spread(fnv32(key))
|
||||
index := dict.spread(key)
|
||||
indexMap[index] = struct{}{}
|
||||
}
|
||||
indices := make([]uint32, 0, len(indexMap))
|
||||
@@ -403,7 +409,7 @@ func (dict *ConcurrentDict) RWLocks(writeKeys []string, readKeys []string) {
|
||||
indices := dict.toLockIndices(keys, false)
|
||||
writeIndexSet := make(map[uint32]struct{})
|
||||
for _, wKey := range writeKeys {
|
||||
idx := dict.spread(fnv32(wKey))
|
||||
idx := dict.spread(wKey)
|
||||
writeIndexSet[idx] = struct{}{}
|
||||
}
|
||||
for _, index := range indices {
|
||||
@@ -423,7 +429,7 @@ func (dict *ConcurrentDict) RWUnLocks(writeKeys []string, readKeys []string) {
|
||||
indices := dict.toLockIndices(keys, true)
|
||||
writeIndexSet := make(map[uint32]struct{})
|
||||
for _, wKey := range writeKeys {
|
||||
idx := dict.spread(fnv32(wKey))
|
||||
idx := dict.spread(wKey)
|
||||
writeIndexSet[idx] = struct{}{}
|
||||
}
|
||||
for _, index := range indices {
|
||||
|
||||
@@ -21,6 +21,17 @@ func Make(members ...string) *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
|
||||
func (set *Set) Add(val string) int {
|
||||
return set.dict.Put(val, nil)
|
||||
|
||||
19
go.mod
19
go.mod
@@ -1,7 +1,24 @@
|
||||
module github.com/hdt3213/godis
|
||||
|
||||
go 1.17
|
||||
go 1.18
|
||||
|
||||
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
|
||||
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.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-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.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
|
||||
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/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/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/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/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8=
|
||||
github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o=
|
||||
github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
|
||||
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.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.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.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.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=
|
||||
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 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.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
|
||||
gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
|
||||
rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4=
|
||||
|
||||
@@ -63,6 +63,10 @@ func MakeClient(addr string) (*Client, error) {
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (client *Client) RemoteAddress() string {
|
||||
return client.addr
|
||||
}
|
||||
|
||||
// Start starts asynchronous goroutines
|
||||
func (client *Client) Start() {
|
||||
client.ticker = time.NewTicker(10 * time.Second)
|
||||
|
||||
@@ -2,8 +2,10 @@ package protocol
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"errors"
|
||||
"strconv"
|
||||
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
)
|
||||
|
||||
var (
|
||||
@@ -176,6 +178,17 @@ func IsErrorReply(reply redis.Reply) bool {
|
||||
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
|
||||
func (r *StandardErrReply) ToBytes() []byte {
|
||||
return []byte("-" + r.Status + CRLF)
|
||||
|
||||
Reference in New Issue
Block a user