mirror of
https://github.com/HDT3213/godis.git
synced 2025-10-06 01:07:06 +08:00
replication slave side
This commit is contained in:
@@ -14,12 +14,16 @@ import (
|
||||
type ServerProperties struct {
|
||||
Bind string `cfg:"bind"`
|
||||
Port int `cfg:"port"`
|
||||
AppendOnly bool `cfg:"appendOnly"`
|
||||
AppendFilename string `cfg:"appendFilename"`
|
||||
AppendOnly bool `cfg:"appendonly"`
|
||||
AppendFilename string `cfg:"appendfilename"`
|
||||
MaxClients int `cfg:"maxclients"`
|
||||
RequirePass string `cfg:"requirepass"`
|
||||
Databases int `cfg:"databases"`
|
||||
RDBFilename string `cfg:"dbfilename"`
|
||||
MasterAuth string `cfg:"masterauth""`
|
||||
SlaveAnnouncePort int `cfg:"slave-announce-port"`
|
||||
SlaveAnnounceIP string `cfg:"slave-announce-ip"`
|
||||
ReplTimeout int `cfg:"repl-timeout"`
|
||||
|
||||
Peers []string `cfg:"peers"`
|
||||
Self string `cfg:"self"`
|
||||
|
@@ -129,11 +129,11 @@ func execCopyTo(db *DB, args [][]byte) redis.Reply {
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("DumpKey", execDumpKey, writeAllKeys, undoDel, 2)
|
||||
RegisterCommand("ExistIn", execExistIn, readAllKeys, nil, -1)
|
||||
RegisterCommand("RenameFrom", execRenameFrom, readFirstKey, nil, 2)
|
||||
RegisterCommand("RenameTo", execRenameTo, writeFirstKey, rollbackFirstKey, 4)
|
||||
RegisterCommand("RenameNxTo", execRenameTo, writeFirstKey, rollbackFirstKey, 4)
|
||||
RegisterCommand("CopyFrom", execCopyFrom, readFirstKey, nil, 2)
|
||||
RegisterCommand("CopyTo", execCopyTo, writeFirstKey, rollbackFirstKey, 5)
|
||||
RegisterCommand("DumpKey", execDumpKey, writeAllKeys, undoDel, 2, flagReadOnly)
|
||||
RegisterCommand("ExistIn", execExistIn, readAllKeys, nil, -1, flagReadOnly)
|
||||
RegisterCommand("RenameFrom", execRenameFrom, readFirstKey, nil, 2, flagWrite)
|
||||
RegisterCommand("RenameTo", execRenameTo, writeFirstKey, rollbackFirstKey, 4, flagWrite)
|
||||
RegisterCommand("RenameNxTo", execRenameTo, writeFirstKey, rollbackFirstKey, 4, flagWrite)
|
||||
RegisterCommand("CopyFrom", execCopyFrom, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("CopyTo", execCopyTo, writeFirstKey, rollbackFirstKey, 5, flagWrite)
|
||||
}
|
||||
|
@@ -9,10 +9,12 @@ import (
|
||||
"github.com/hdt3213/godis/lib/logger"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/pubsub"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"runtime/debug"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
)
|
||||
|
||||
@@ -24,6 +26,11 @@ type MultiDB struct {
|
||||
hub *pubsub.Hub
|
||||
// handle aof persistence
|
||||
aofHandler *aof.Handler
|
||||
|
||||
// store master node address
|
||||
slaveOf string
|
||||
role int32
|
||||
replication *replicationStatus
|
||||
}
|
||||
|
||||
// NewStandaloneServer creates a standalone redis server, with multi database and all other funtions
|
||||
@@ -59,8 +66,11 @@ func NewStandaloneServer() *MultiDB {
|
||||
}
|
||||
if config.Properties.RDBFilename != "" && !validAof {
|
||||
// load rdb
|
||||
loadRdb(mdb)
|
||||
loadRdbFile(mdb)
|
||||
}
|
||||
mdb.replication = initReplStatus()
|
||||
mdb.startReplCron()
|
||||
mdb.role = masterRole // The initialization process does not require atomicity
|
||||
return mdb
|
||||
}
|
||||
|
||||
@@ -92,6 +102,25 @@ func (mdb *MultiDB) Exec(c redis.Connection, cmdLine [][]byte) (result redis.Rep
|
||||
if !isAuthenticated(c) {
|
||||
return protocol.MakeErrReply("NOAUTH Authentication required")
|
||||
}
|
||||
if cmdName == "slaveof" {
|
||||
if c != nil && c.InMultiState() {
|
||||
return protocol.MakeErrReply("cannot use slave of database within multi")
|
||||
}
|
||||
if len(cmdLine) != 3 {
|
||||
return protocol.MakeArgNumErrReply("SLAVEOF")
|
||||
}
|
||||
return mdb.execSlaveOf(c, cmdLine[1:])
|
||||
}
|
||||
|
||||
// read only slave
|
||||
role := atomic.LoadInt32(&mdb.role)
|
||||
if role == slaveRole &&
|
||||
c.GetRole() != connection.ReplicationRecvCli {
|
||||
// only allow read only command, forbid all special commands except `auth` and `slaveof`
|
||||
if !isReadOnlyCommand(cmdName) {
|
||||
return protocol.MakeErrReply("READONLY You can't write against a read only slave.")
|
||||
}
|
||||
}
|
||||
|
||||
// special commands which cannot execute within transaction
|
||||
if cmdName == "subscribe" {
|
||||
@@ -146,6 +175,8 @@ func (mdb *MultiDB) AfterClientClose(c redis.Connection) {
|
||||
|
||||
// Close graceful shutdown database
|
||||
func (mdb *MultiDB) Close() {
|
||||
// stop replication first
|
||||
mdb.replication.close()
|
||||
if mdb.aofHandler != nil {
|
||||
mdb.aofHandler.Close()
|
||||
}
|
||||
|
@@ -262,10 +262,10 @@ func geoRadius0(sortedSet *sortedset.SortedSet, lat float64, lng float64, radius
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("GeoAdd", execGeoAdd, writeFirstKey, undoGeoAdd, -5)
|
||||
RegisterCommand("GeoPos", execGeoPos, readFirstKey, nil, -2)
|
||||
RegisterCommand("GeoDist", execGeoDist, readFirstKey, nil, -4)
|
||||
RegisterCommand("GeoHash", execGeoHash, readFirstKey, nil, -2)
|
||||
RegisterCommand("GeoRadius", execGeoRadius, readFirstKey, nil, -6)
|
||||
RegisterCommand("GeoRadiusByMember", execGeoRadiusByMember, readFirstKey, nil, -5)
|
||||
RegisterCommand("GeoAdd", execGeoAdd, writeFirstKey, undoGeoAdd, -5, flagWrite)
|
||||
RegisterCommand("GeoPos", execGeoPos, readFirstKey, nil, -2, flagReadOnly)
|
||||
RegisterCommand("GeoDist", execGeoDist, readFirstKey, nil, -4, flagReadOnly)
|
||||
RegisterCommand("GeoHash", execGeoHash, readFirstKey, nil, -2, flagReadOnly)
|
||||
RegisterCommand("GeoRadius", execGeoRadius, readFirstKey, nil, -6, flagReadOnly)
|
||||
RegisterCommand("GeoRadiusByMember", execGeoRadiusByMember, readFirstKey, nil, -5, flagReadOnly)
|
||||
}
|
||||
|
@@ -498,20 +498,20 @@ func execHRandField(db *DB, args [][]byte) redis.Reply {
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("HSet", execHSet, writeFirstKey, undoHSet, 4)
|
||||
RegisterCommand("HSetNX", execHSetNX, writeFirstKey, undoHSet, 4)
|
||||
RegisterCommand("HGet", execHGet, readFirstKey, nil, 3)
|
||||
RegisterCommand("HExists", execHExists, readFirstKey, nil, 3)
|
||||
RegisterCommand("HDel", execHDel, writeFirstKey, undoHDel, -3)
|
||||
RegisterCommand("HLen", execHLen, readFirstKey, nil, 2)
|
||||
RegisterCommand("HStrlen", execHStrlen, readFirstKey, nil, 3)
|
||||
RegisterCommand("HMSet", execHMSet, writeFirstKey, undoHMSet, -4)
|
||||
RegisterCommand("HMGet", execHMGet, readFirstKey, nil, -3)
|
||||
RegisterCommand("HGet", execHGet, readFirstKey, nil, -3)
|
||||
RegisterCommand("HKeys", execHKeys, readFirstKey, nil, 2)
|
||||
RegisterCommand("HVals", execHVals, readFirstKey, nil, 2)
|
||||
RegisterCommand("HGetAll", execHGetAll, readFirstKey, nil, 2)
|
||||
RegisterCommand("HIncrBy", execHIncrBy, writeFirstKey, undoHIncr, 4)
|
||||
RegisterCommand("HIncrByFloat", execHIncrByFloat, writeFirstKey, undoHIncr, 4)
|
||||
RegisterCommand("HRandField", execHRandField, readFirstKey, nil, -2)
|
||||
RegisterCommand("HSet", execHSet, writeFirstKey, undoHSet, 4, flagWrite)
|
||||
RegisterCommand("HSetNX", execHSetNX, writeFirstKey, undoHSet, 4, flagWrite)
|
||||
RegisterCommand("HGet", execHGet, readFirstKey, nil, 3, flagReadOnly)
|
||||
RegisterCommand("HExists", execHExists, readFirstKey, nil, 3, flagReadOnly)
|
||||
RegisterCommand("HDel", execHDel, writeFirstKey, undoHDel, -3, flagWrite)
|
||||
RegisterCommand("HLen", execHLen, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("HStrlen", execHStrlen, readFirstKey, nil, 3, flagReadOnly)
|
||||
RegisterCommand("HMSet", execHMSet, writeFirstKey, undoHMSet, -4, flagWrite)
|
||||
RegisterCommand("HMGet", execHMGet, readFirstKey, nil, -3, flagReadOnly)
|
||||
RegisterCommand("HGet", execHGet, readFirstKey, nil, -3, flagReadOnly)
|
||||
RegisterCommand("HKeys", execHKeys, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("HVals", execHVals, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("HGetAll", execHGetAll, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("HIncrBy", execHIncrBy, writeFirstKey, undoHIncr, 4, flagWrite)
|
||||
RegisterCommand("HIncrByFloat", execHIncrByFloat, writeFirstKey, undoHIncr, 4, flagWrite)
|
||||
RegisterCommand("HRandField", execHRandField, readFirstKey, nil, -2, flagReadOnly)
|
||||
}
|
||||
|
@@ -372,17 +372,17 @@ func execCopy(mdb *MultiDB, conn redis.Connection, args [][]byte) redis.Reply {
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("Del", execDel, writeAllKeys, undoDel, -2)
|
||||
RegisterCommand("Expire", execExpire, writeFirstKey, undoExpire, 3)
|
||||
RegisterCommand("ExpireAt", execExpireAt, writeFirstKey, undoExpire, 3)
|
||||
RegisterCommand("PExpire", execPExpire, writeFirstKey, undoExpire, 3)
|
||||
RegisterCommand("PExpireAt", execPExpireAt, writeFirstKey, undoExpire, 3)
|
||||
RegisterCommand("TTL", execTTL, readFirstKey, nil, 2)
|
||||
RegisterCommand("PTTL", execPTTL, readFirstKey, nil, 2)
|
||||
RegisterCommand("Persist", execPersist, writeFirstKey, undoExpire, 2)
|
||||
RegisterCommand("Exists", execExists, readAllKeys, nil, -2)
|
||||
RegisterCommand("Type", execType, readFirstKey, nil, 2)
|
||||
RegisterCommand("Rename", execRename, prepareRename, undoRename, 3)
|
||||
RegisterCommand("RenameNx", execRenameNx, prepareRename, undoRename, 3)
|
||||
RegisterCommand("Keys", execKeys, noPrepare, nil, 2)
|
||||
RegisterCommand("Del", execDel, writeAllKeys, undoDel, -2, flagWrite)
|
||||
RegisterCommand("Expire", execExpire, writeFirstKey, undoExpire, 3, flagWrite)
|
||||
RegisterCommand("ExpireAt", execExpireAt, writeFirstKey, undoExpire, 3, flagWrite)
|
||||
RegisterCommand("PExpire", execPExpire, writeFirstKey, undoExpire, 3, flagWrite)
|
||||
RegisterCommand("PExpireAt", execPExpireAt, writeFirstKey, undoExpire, 3, flagWrite)
|
||||
RegisterCommand("TTL", execTTL, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("PTTL", execPTTL, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("Persist", execPersist, writeFirstKey, undoExpire, 2, flagWrite)
|
||||
RegisterCommand("Exists", execExists, readAllKeys, nil, -2, flagReadOnly)
|
||||
RegisterCommand("Type", execType, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("Rename", execRename, prepareRename, undoRename, 3, flagReadOnly)
|
||||
RegisterCommand("RenameNx", execRenameNx, prepareRename, undoRename, 3, flagReadOnly)
|
||||
RegisterCommand("Keys", execKeys, noPrepare, nil, 2, flagReadOnly)
|
||||
}
|
||||
|
@@ -505,16 +505,16 @@ func execRPushX(db *DB, args [][]byte) redis.Reply {
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("LPush", execLPush, writeFirstKey, undoLPush, -3)
|
||||
RegisterCommand("LPushX", execLPushX, writeFirstKey, undoLPush, -3)
|
||||
RegisterCommand("RPush", execRPush, writeFirstKey, undoRPush, -3)
|
||||
RegisterCommand("RPushX", execRPushX, writeFirstKey, undoRPush, -3)
|
||||
RegisterCommand("LPop", execLPop, writeFirstKey, undoLPop, 2)
|
||||
RegisterCommand("RPop", execRPop, writeFirstKey, undoRPop, 2)
|
||||
RegisterCommand("RPopLPush", execRPopLPush, prepareRPopLPush, undoRPopLPush, 3)
|
||||
RegisterCommand("LRem", execLRem, writeFirstKey, rollbackFirstKey, 4)
|
||||
RegisterCommand("LLen", execLLen, readFirstKey, nil, 2)
|
||||
RegisterCommand("LIndex", execLIndex, readFirstKey, nil, 3)
|
||||
RegisterCommand("LSet", execLSet, writeFirstKey, undoLSet, 4)
|
||||
RegisterCommand("LRange", execLRange, readFirstKey, nil, 4)
|
||||
RegisterCommand("LPush", execLPush, writeFirstKey, undoLPush, -3, flagWrite)
|
||||
RegisterCommand("LPushX", execLPushX, writeFirstKey, undoLPush, -3, flagWrite)
|
||||
RegisterCommand("RPush", execRPush, writeFirstKey, undoRPush, -3, flagWrite)
|
||||
RegisterCommand("RPushX", execRPushX, writeFirstKey, undoRPush, -3, flagWrite)
|
||||
RegisterCommand("LPop", execLPop, writeFirstKey, undoLPop, 2, flagWrite)
|
||||
RegisterCommand("RPop", execRPop, writeFirstKey, undoRPop, 2, flagWrite)
|
||||
RegisterCommand("RPopLPush", execRPopLPush, prepareRPopLPush, undoRPopLPush, 3, flagWrite)
|
||||
RegisterCommand("LRem", execLRem, writeFirstKey, rollbackFirstKey, 4, flagWrite)
|
||||
RegisterCommand("LLen", execLLen, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("LIndex", execLIndex, readFirstKey, nil, 3, flagReadOnly)
|
||||
RegisterCommand("LSet", execLSet, writeFirstKey, undoLSet, 4, flagWrite)
|
||||
RegisterCommand("LRange", execLRange, readFirstKey, nil, 4, flagReadOnly)
|
||||
}
|
||||
|
@@ -7,21 +7,30 @@ import (
|
||||
SortedSet "github.com/hdt3213/godis/datastruct/sortedset"
|
||||
"github.com/hdt3213/godis/interface/database"
|
||||
"github.com/hdt3213/godis/lib/logger"
|
||||
"github.com/hdt3213/rdb/core"
|
||||
rdb "github.com/hdt3213/rdb/parser"
|
||||
"os"
|
||||
)
|
||||
|
||||
func loadRdb(mdb *MultiDB) {
|
||||
func loadRdbFile(mdb *MultiDB) {
|
||||
rdbFile, err := os.Open(config.Properties.RDBFilename)
|
||||
if err != nil {
|
||||
logger.Error("open rdb file failed")
|
||||
logger.Error("open rdb file failed " + err.Error())
|
||||
return
|
||||
}
|
||||
defer func() {
|
||||
_ = rdbFile.Close()
|
||||
}()
|
||||
decoder := rdb.NewDecoder(rdbFile)
|
||||
err = decoder.Parse(func(o rdb.RedisObject) bool {
|
||||
err = dumpRDB(decoder, mdb)
|
||||
if err != nil {
|
||||
logger.Error("dump rdb file failed " + err.Error())
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
func dumpRDB(dec *core.Decoder, mdb *MultiDB) error {
|
||||
return dec.Parse(func(o rdb.RedisObject) bool {
|
||||
db := mdb.selectDB(o.GetDBIndex())
|
||||
switch o.GetType() {
|
||||
case rdb.StringType:
|
||||
|
401
database/replication.go
Normal file
401
database/replication.go
Normal file
@@ -0,0 +1,401 @@
|
||||
package database
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"github.com/hdt3213/godis/config"
|
||||
"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/parser"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
rdb "github.com/hdt3213/rdb/parser"
|
||||
"net"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
masterRole = iota
|
||||
slaveRole
|
||||
)
|
||||
|
||||
type replicationStatus struct {
|
||||
mutex sync.Mutex
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
modCount int32 // execSlaveOf() or reconnect will cause modCount++
|
||||
|
||||
masterHost string
|
||||
masterPort int
|
||||
|
||||
masterConn net.Conn
|
||||
masterChan <-chan *parser.Payload
|
||||
replId string
|
||||
replOffset int64
|
||||
lastRecvTime time.Time
|
||||
running sync.WaitGroup
|
||||
}
|
||||
|
||||
func initReplStatus() *replicationStatus {
|
||||
repl := &replicationStatus{}
|
||||
// start cron
|
||||
return repl
|
||||
}
|
||||
|
||||
func (mdb *MultiDB) startReplCron() {
|
||||
go func() {
|
||||
defer func() {
|
||||
if err := recover(); err != nil {
|
||||
logger.Error("panic", err)
|
||||
}
|
||||
}()
|
||||
ticker := time.Tick(time.Second)
|
||||
for range ticker {
|
||||
mdb.slaveCron()
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (mdb *MultiDB) execSlaveOf(c redis.Connection, args [][]byte) redis.Reply {
|
||||
if strings.ToLower(string(args[0])) == "no" &&
|
||||
strings.ToLower(string(args[1])) == "one" {
|
||||
mdb.slaveOfNone()
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
host := string(args[0])
|
||||
port, err := strconv.Atoi(string(args[1]))
|
||||
if err != nil {
|
||||
return protocol.MakeErrReply("ERR value is not an integer or out of range")
|
||||
}
|
||||
mdb.replication.mutex.Lock()
|
||||
atomic.StoreInt32(&mdb.role, slaveRole)
|
||||
mdb.replication.masterHost = host
|
||||
mdb.replication.masterPort = port
|
||||
// use buffered channel in case receiver goroutine exited before controller send stop signal
|
||||
atomic.AddInt32(&mdb.replication.modCount, 1)
|
||||
mdb.replication.mutex.Unlock()
|
||||
go mdb.syncWithMaster()
|
||||
return protocol.MakeOkReply()
|
||||
}
|
||||
|
||||
func (mdb *MultiDB) slaveOfNone() {
|
||||
mdb.replication.mutex.Lock()
|
||||
defer mdb.replication.mutex.Unlock()
|
||||
mdb.replication.masterHost = ""
|
||||
mdb.replication.masterPort = 0
|
||||
mdb.replication.replId = ""
|
||||
mdb.replication.replOffset = -1
|
||||
mdb.replication.stopSlaveWithMutex()
|
||||
}
|
||||
|
||||
// stopSlaveWithMutex stops in-progress connectWithMaster/fullSync/receiveAOF
|
||||
// invoker should have replication mutex
|
||||
func (repl *replicationStatus) stopSlaveWithMutex() {
|
||||
// update modCount to stop connectWithMaster and fullSync
|
||||
atomic.AddInt32(&repl.modCount, 1)
|
||||
// send cancel to receiveAOF
|
||||
if repl.cancel != nil {
|
||||
repl.cancel()
|
||||
repl.running.Wait()
|
||||
}
|
||||
repl.ctx = context.Background()
|
||||
repl.cancel = nil
|
||||
if repl.masterConn != nil {
|
||||
_ = repl.masterConn.Close() // parser.ParseStream will close masterChan
|
||||
}
|
||||
repl.masterConn = nil
|
||||
repl.masterChan = nil
|
||||
}
|
||||
|
||||
func (repl *replicationStatus) close() error {
|
||||
repl.mutex.Lock()
|
||||
defer repl.mutex.Unlock()
|
||||
repl.stopSlaveWithMutex()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mdb *MultiDB) syncWithMaster() {
|
||||
defer func() {
|
||||
if err := recover(); err != nil {
|
||||
logger.Error(err)
|
||||
}
|
||||
}()
|
||||
mdb.replication.mutex.Lock()
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
mdb.replication.ctx = ctx
|
||||
mdb.replication.cancel = cancel
|
||||
mdb.replication.mutex.Unlock()
|
||||
err := mdb.connectWithMaster()
|
||||
if err != nil {
|
||||
// full sync failed, abort
|
||||
return
|
||||
}
|
||||
err = mdb.doPsync()
|
||||
if err != nil {
|
||||
// full sync failed, abort
|
||||
return
|
||||
}
|
||||
err = mdb.receiveAOF()
|
||||
if err != nil {
|
||||
// full sync failed, abort
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
func (mdb *MultiDB) connectWithMaster() error {
|
||||
modCount := atomic.LoadInt32(&mdb.replication.modCount)
|
||||
addr := mdb.replication.masterHost + ":" + strconv.Itoa(mdb.replication.masterPort)
|
||||
conn, err := net.Dial("tcp", addr)
|
||||
if err != nil {
|
||||
mdb.slaveOfNone() // abort
|
||||
return errors.New("connect master failed " + err.Error())
|
||||
}
|
||||
masterChan := parser.ParseStream(conn)
|
||||
|
||||
// ping
|
||||
pingCmdLine := utils.ToCmdLine("ping")
|
||||
pingReq := protocol.MakeMultiBulkReply(pingCmdLine)
|
||||
_, err = conn.Write(pingReq.ToBytes())
|
||||
if err != nil {
|
||||
return errors.New("send failed " + err.Error())
|
||||
}
|
||||
pingResp := <-masterChan
|
||||
if pingResp.Err != nil {
|
||||
return errors.New("read response failed: " + pingResp.Err.Error())
|
||||
}
|
||||
switch reply := pingResp.Data.(type) {
|
||||
case *protocol.StandardErrReply:
|
||||
if !strings.HasPrefix(reply.Error(), "NOAUTH") &&
|
||||
!strings.HasPrefix(reply.Error(), "NOPERM") &&
|
||||
!strings.HasPrefix(reply.Error(), "ERR operation not permitted") {
|
||||
logger.Error("Error reply to PING from master: " + string(reply.ToBytes()))
|
||||
mdb.slaveOfNone() // abort
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// just to reduce duplication of code
|
||||
sendCmdToMaster := func(conn net.Conn, cmdLine CmdLine, masterChan <-chan *parser.Payload) error {
|
||||
req := protocol.MakeMultiBulkReply(cmdLine)
|
||||
_, err := conn.Write(req.ToBytes())
|
||||
if err != nil {
|
||||
mdb.slaveOfNone() // abort
|
||||
return errors.New("send failed " + err.Error())
|
||||
}
|
||||
resp := <-masterChan
|
||||
if resp.Err != nil {
|
||||
mdb.slaveOfNone() // abort
|
||||
return errors.New("read response failed: " + resp.Err.Error())
|
||||
}
|
||||
if !protocol.IsOKReply(resp.Data) {
|
||||
mdb.slaveOfNone() // abort
|
||||
return errors.New("unexpected auth response: " + string(resp.Data.ToBytes()))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// auth
|
||||
if config.Properties.MasterAuth != "" {
|
||||
authCmdLine := utils.ToCmdLine("auth", config.Properties.MasterAuth)
|
||||
err = sendCmdToMaster(conn, authCmdLine, masterChan)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// announce port
|
||||
var port int
|
||||
if config.Properties.SlaveAnnouncePort != 0 {
|
||||
port = config.Properties.SlaveAnnouncePort
|
||||
} else {
|
||||
port = config.Properties.Port
|
||||
}
|
||||
portCmdLine := utils.ToCmdLine("REPLCONF", "listening-port", strconv.Itoa(port))
|
||||
err = sendCmdToMaster(conn, portCmdLine, masterChan)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// announce ip
|
||||
if config.Properties.SlaveAnnounceIP != "" {
|
||||
ipCmdLine := utils.ToCmdLine("REPLCONF", "ip-address", config.Properties.SlaveAnnounceIP)
|
||||
err = sendCmdToMaster(conn, ipCmdLine, masterChan)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// announce capacity
|
||||
capaCmdLine := utils.ToCmdLine("REPLCONF", "capa", "psync2")
|
||||
err = sendCmdToMaster(conn, capaCmdLine, masterChan)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// update connection
|
||||
mdb.replication.mutex.Lock()
|
||||
if mdb.replication.modCount != modCount {
|
||||
// replication conf changed during connecting and waiting mutex
|
||||
return nil
|
||||
}
|
||||
mdb.replication.masterConn = conn
|
||||
mdb.replication.masterChan = masterChan
|
||||
mdb.replication.mutex.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
// doPsync send psync to master and sync repl-id then load rdb
|
||||
func (mdb *MultiDB) doPsync() error {
|
||||
modCount := atomic.LoadInt32(&mdb.replication.modCount)
|
||||
psyncCmdLine := utils.ToCmdLine("psync", "?", "-1")
|
||||
psyncReq := protocol.MakeMultiBulkReply(psyncCmdLine)
|
||||
_, err := mdb.replication.masterConn.Write(psyncReq.ToBytes())
|
||||
if err != nil {
|
||||
return errors.New("send failed " + err.Error())
|
||||
}
|
||||
ch := mdb.replication.masterChan
|
||||
psyncPayload1 := <-ch
|
||||
if psyncPayload1.Err != nil {
|
||||
return errors.New("read response failed: " + psyncPayload1.Err.Error())
|
||||
}
|
||||
psyncHeader, ok := psyncPayload1.Data.(*protocol.StatusReply)
|
||||
if !ok {
|
||||
return errors.New("illegal payload header: " + string(psyncPayload1.Data.ToBytes()))
|
||||
}
|
||||
headers := strings.Split(psyncHeader.Status, " ")
|
||||
if len(headers) != 3 {
|
||||
return errors.New("illegal payload header: " + psyncHeader.Status)
|
||||
}
|
||||
logger.Info("receive psync header from master")
|
||||
psyncPayload2 := <-ch
|
||||
if psyncPayload2.Err != nil {
|
||||
return errors.New("read response failed: " + psyncPayload2.Err.Error())
|
||||
}
|
||||
psyncData, ok := psyncPayload2.Data.(*protocol.BulkReply)
|
||||
if !ok {
|
||||
return errors.New("illegal payload header: " + string(psyncPayload2.Data.ToBytes()))
|
||||
}
|
||||
logger.Info(fmt.Sprintf("receive %d bytes of rdb from master", len(psyncData.Arg)))
|
||||
rdbDec := rdb.NewDecoder(bytes.NewReader(psyncData.Arg))
|
||||
rdbHolder := MakeBasicMultiDB()
|
||||
err = dumpRDB(rdbDec, rdbHolder)
|
||||
if err != nil {
|
||||
return errors.New("dump rdb failed: " + err.Error())
|
||||
}
|
||||
|
||||
mdb.replication.mutex.Lock()
|
||||
defer mdb.replication.mutex.Unlock()
|
||||
if mdb.replication.modCount != modCount {
|
||||
// replication conf changed during connecting and waiting mutex
|
||||
return nil
|
||||
}
|
||||
mdb.replication.replId = headers[1]
|
||||
mdb.replication.replOffset, err = strconv.ParseInt(headers[2], 10, 64)
|
||||
if err != nil {
|
||||
return errors.New("get illegal repl offset: " + headers[2])
|
||||
}
|
||||
logger.Info("full resync from master: " + mdb.replication.replId)
|
||||
logger.Info("current offset:", mdb.replication.replOffset)
|
||||
for i, newDB := range rdbHolder.dbSet {
|
||||
oldDB := mdb.selectDB(i)
|
||||
oldDB.Load(newDB)
|
||||
}
|
||||
// there is no CRLF between RDB and following AOF, reset stream to avoid parser error
|
||||
mdb.replication.masterChan = parser.ParseStream(mdb.replication.masterConn)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mdb *MultiDB) receiveAOF() error {
|
||||
conn := connection.NewConn(mdb.replication.masterConn)
|
||||
conn.SetRole(connection.ReplicationRecvCli)
|
||||
mdb.replication.mutex.Lock()
|
||||
modCount := mdb.replication.modCount
|
||||
done := mdb.replication.ctx.Done()
|
||||
mdb.replication.mutex.Unlock()
|
||||
if done == nil {
|
||||
// stopSlaveWithMutex() is called during waiting mutex
|
||||
return nil
|
||||
}
|
||||
mdb.replication.running.Add(1)
|
||||
defer mdb.replication.running.Done()
|
||||
for {
|
||||
select {
|
||||
case payload, open := <-mdb.replication.masterChan:
|
||||
if !open {
|
||||
return errors.New("master channel unexpected close")
|
||||
}
|
||||
if payload.Err != nil {
|
||||
return payload.Err
|
||||
}
|
||||
cmdLine, ok := payload.Data.(*protocol.MultiBulkReply)
|
||||
if !ok {
|
||||
return errors.New("unexpected payload: " + string(payload.Data.ToBytes()))
|
||||
}
|
||||
mdb.replication.mutex.Lock()
|
||||
if mdb.replication.modCount != modCount {
|
||||
// replication conf changed during connecting and waiting mutex
|
||||
return nil
|
||||
}
|
||||
mdb.Exec(conn, cmdLine.Args)
|
||||
// todo: directly get size from socket
|
||||
n := len(cmdLine.ToBytes())
|
||||
mdb.replication.replOffset += int64(n)
|
||||
logger.Info(fmt.Sprintf("receive %d bytes from master, current offset %d",
|
||||
n, mdb.replication.replOffset))
|
||||
mdb.replication.mutex.Unlock()
|
||||
case <-done:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (mdb *MultiDB) slaveCron() {
|
||||
repl := mdb.replication
|
||||
if repl.masterConn == nil {
|
||||
return
|
||||
}
|
||||
replTimeout := 60 * time.Second
|
||||
if config.Properties.ReplTimeout != 0 {
|
||||
replTimeout = time.Duration(config.Properties.ReplTimeout) * time.Second
|
||||
}
|
||||
minLastRecvTime := time.Now().Add(-replTimeout)
|
||||
if repl.lastRecvTime.Before(minLastRecvTime) {
|
||||
// reconnect with master
|
||||
err := mdb.reconnectWithMaster()
|
||||
if err != nil {
|
||||
logger.Error("send failed " + err.Error())
|
||||
}
|
||||
return
|
||||
}
|
||||
// send ack to master
|
||||
err := repl.sendAck2Master()
|
||||
if err != nil {
|
||||
logger.Error("send failed " + err.Error())
|
||||
}
|
||||
}
|
||||
|
||||
// Send a REPLCONF ACK command to the master to inform it about the current processed offset
|
||||
func (repl *replicationStatus) sendAck2Master() error {
|
||||
psyncCmdLine := utils.ToCmdLine("REPLCONF", "ACK",
|
||||
strconv.FormatInt(repl.replOffset, 10))
|
||||
psyncReq := protocol.MakeMultiBulkReply(psyncCmdLine)
|
||||
_, err := repl.masterConn.Write(psyncReq.ToBytes())
|
||||
//logger.Info("send ack to master")
|
||||
return err
|
||||
}
|
||||
|
||||
func (mdb *MultiDB) reconnectWithMaster() error {
|
||||
logger.Info("reconnecting with master")
|
||||
mdb.replication.mutex.Lock()
|
||||
defer mdb.replication.mutex.Unlock()
|
||||
mdb.replication.stopSlaveWithMutex()
|
||||
go mdb.syncWithMaster()
|
||||
return nil
|
||||
}
|
135
database/replication_test.go
Normal file
135
database/replication_test.go
Normal file
@@ -0,0 +1,135 @@
|
||||
package database
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"github.com/hdt3213/godis/config"
|
||||
"github.com/hdt3213/godis/lib/utils"
|
||||
"github.com/hdt3213/godis/redis/client"
|
||||
"github.com/hdt3213/godis/redis/connection"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"github.com/hdt3213/godis/redis/protocol/asserts"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestReplication(t *testing.T) {
|
||||
mdb := &MultiDB{}
|
||||
mdb.dbSet = make([]*DB, 16)
|
||||
for i := range mdb.dbSet {
|
||||
singleDB := makeDB()
|
||||
singleDB.index = i
|
||||
mdb.dbSet[i] = singleDB
|
||||
}
|
||||
mdb.replication = initReplStatus()
|
||||
masterCli, err := client.MakeClient("127.0.0.1:6379")
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
masterCli.Start()
|
||||
|
||||
// sync with master
|
||||
ret := masterCli.Send(utils.ToCmdLine("set", "1", "1"))
|
||||
asserts.AssertStatusReply(t, ret, "OK")
|
||||
conn := &connection.FakeConn{}
|
||||
ret = mdb.Exec(conn, utils.ToCmdLine("SLAVEOF", "127.0.0.1", "6379"))
|
||||
asserts.AssertStatusReply(t, ret, "OK")
|
||||
success := false
|
||||
for i := 0; i < 30; i++ {
|
||||
// wait for sync
|
||||
time.Sleep(time.Second)
|
||||
ret = mdb.Exec(conn, utils.ToCmdLine("GET", "1"))
|
||||
bulkRet, ok := ret.(*protocol.BulkReply)
|
||||
if ok {
|
||||
if bytes.Equal(bulkRet.Arg, []byte("1")) {
|
||||
success = true
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !success {
|
||||
t.Error("sync failed")
|
||||
return
|
||||
}
|
||||
|
||||
// receive aof
|
||||
ret = masterCli.Send(utils.ToCmdLine("set", "1", "2"))
|
||||
asserts.AssertStatusReply(t, ret, "OK")
|
||||
success = false
|
||||
for i := 0; i < 10; i++ {
|
||||
// wait for sync
|
||||
time.Sleep(time.Second)
|
||||
ret = mdb.Exec(conn, utils.ToCmdLine("GET", "1"))
|
||||
bulkRet, ok := ret.(*protocol.BulkReply)
|
||||
if ok {
|
||||
if bytes.Equal(bulkRet.Arg, []byte("2")) {
|
||||
success = true
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !success {
|
||||
t.Error("sync failed")
|
||||
return
|
||||
}
|
||||
err = mdb.replication.sendAck2Master()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
}
|
||||
|
||||
// test reconnect
|
||||
config.Properties.ReplTimeout = 1
|
||||
_ = mdb.replication.masterConn.Close()
|
||||
ret = masterCli.Send(utils.ToCmdLine("set", "1", "3"))
|
||||
asserts.AssertStatusReply(t, ret, "OK")
|
||||
mdb.slaveCron()
|
||||
success = false
|
||||
for i := 0; i < 10; i++ {
|
||||
// wait for sync
|
||||
time.Sleep(time.Second)
|
||||
ret = mdb.Exec(conn, utils.ToCmdLine("GET", "1"))
|
||||
bulkRet, ok := ret.(*protocol.BulkReply)
|
||||
if ok {
|
||||
if bytes.Equal(bulkRet.Arg, []byte("3")) {
|
||||
success = true
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !success {
|
||||
t.Error("sync failed")
|
||||
return
|
||||
}
|
||||
|
||||
// test slave of no one
|
||||
ret = mdb.Exec(conn, utils.ToCmdLine("SLAVEOF", "NO", "ONE"))
|
||||
asserts.AssertStatusReply(t, ret, "OK")
|
||||
ret = masterCli.Send(utils.ToCmdLine("set", "1", "4"))
|
||||
asserts.AssertStatusReply(t, ret, "OK")
|
||||
ret = mdb.Exec(conn, utils.ToCmdLine("GET", "1"))
|
||||
asserts.AssertBulkReply(t, ret, "3")
|
||||
ret = mdb.Exec(conn, utils.ToCmdLine("SLAVEOF", "127.0.0.1", "6379"))
|
||||
asserts.AssertStatusReply(t, ret, "OK")
|
||||
success = false
|
||||
for i := 0; i < 30; i++ {
|
||||
// wait for sync
|
||||
time.Sleep(time.Second)
|
||||
ret = mdb.Exec(conn, utils.ToCmdLine("GET", "1"))
|
||||
bulkRet, ok := ret.(*protocol.BulkReply)
|
||||
if ok {
|
||||
if bytes.Equal(bulkRet.Arg, []byte("4")) {
|
||||
success = true
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !success {
|
||||
t.Error("sync failed")
|
||||
return
|
||||
}
|
||||
|
||||
err = mdb.replication.close()
|
||||
if err != nil {
|
||||
t.Error("cannot close")
|
||||
}
|
||||
}
|
@@ -14,15 +14,30 @@ type command struct {
|
||||
flags int
|
||||
}
|
||||
|
||||
const (
|
||||
flagWrite = 0
|
||||
flagReadOnly = 1
|
||||
)
|
||||
|
||||
// RegisterCommand registers a new command
|
||||
// arity means allowed number of cmdArgs, arity < 0 means len(args) >= -arity.
|
||||
// for example: the arity of `get` is 2, `mget` is -2
|
||||
func RegisterCommand(name string, executor ExecFunc, prepare PreFunc, rollback UndoFunc, arity int) {
|
||||
func RegisterCommand(name string, executor ExecFunc, prepare PreFunc, rollback UndoFunc, arity int, flags int) {
|
||||
name = strings.ToLower(name)
|
||||
cmdTable[name] = &command{
|
||||
executor: executor,
|
||||
prepare: prepare,
|
||||
undo: rollback,
|
||||
arity: arity,
|
||||
flags: flags,
|
||||
}
|
||||
}
|
||||
|
||||
func isReadOnlyCommand(name string) bool {
|
||||
name = strings.ToLower(name)
|
||||
cmd := cmdTable[name]
|
||||
if cmd == nil {
|
||||
return false
|
||||
}
|
||||
return cmd.flags&flagReadOnly > 0
|
||||
}
|
||||
|
@@ -479,17 +479,17 @@ func execSRandMember(db *DB, args [][]byte) redis.Reply {
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("SAdd", execSAdd, writeFirstKey, undoSetChange, -3)
|
||||
RegisterCommand("SIsMember", execSIsMember, readFirstKey, nil, 3)
|
||||
RegisterCommand("SRem", execSRem, writeFirstKey, undoSetChange, -3)
|
||||
RegisterCommand("SPop", execSPop, writeFirstKey, undoSetChange, -2)
|
||||
RegisterCommand("SCard", execSCard, readFirstKey, nil, 2)
|
||||
RegisterCommand("SMembers", execSMembers, readFirstKey, nil, 2)
|
||||
RegisterCommand("SInter", execSInter, prepareSetCalculate, nil, -2)
|
||||
RegisterCommand("SInterStore", execSInterStore, prepareSetCalculateStore, rollbackFirstKey, -3)
|
||||
RegisterCommand("SUnion", execSUnion, prepareSetCalculate, nil, -2)
|
||||
RegisterCommand("SUnionStore", execSUnionStore, prepareSetCalculateStore, rollbackFirstKey, -3)
|
||||
RegisterCommand("SDiff", execSDiff, prepareSetCalculate, nil, -2)
|
||||
RegisterCommand("SDiffStore", execSDiffStore, prepareSetCalculateStore, rollbackFirstKey, -3)
|
||||
RegisterCommand("SRandMember", execSRandMember, readFirstKey, nil, -2)
|
||||
RegisterCommand("SAdd", execSAdd, writeFirstKey, undoSetChange, -3, flagWrite)
|
||||
RegisterCommand("SIsMember", execSIsMember, readFirstKey, nil, 3, flagReadOnly)
|
||||
RegisterCommand("SRem", execSRem, writeFirstKey, undoSetChange, -3, flagWrite)
|
||||
RegisterCommand("SPop", execSPop, writeFirstKey, undoSetChange, -2, flagWrite)
|
||||
RegisterCommand("SCard", execSCard, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("SMembers", execSMembers, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("SInter", execSInter, prepareSetCalculate, nil, -2, flagReadOnly)
|
||||
RegisterCommand("SInterStore", execSInterStore, prepareSetCalculateStore, rollbackFirstKey, -3, flagWrite)
|
||||
RegisterCommand("SUnion", execSUnion, prepareSetCalculate, nil, -2, flagReadOnly)
|
||||
RegisterCommand("SUnionStore", execSUnionStore, prepareSetCalculateStore, rollbackFirstKey, -3, flagWrite)
|
||||
RegisterCommand("SDiff", execSDiff, prepareSetCalculate, nil, -2, flagReadOnly)
|
||||
RegisterCommand("SDiffStore", execSDiffStore, prepareSetCalculateStore, rollbackFirstKey, -3, flagWrite)
|
||||
RegisterCommand("SRandMember", execSRandMember, readFirstKey, nil, -2, flagReadOnly)
|
||||
}
|
||||
|
@@ -226,7 +226,15 @@ func (db *DB) Flush() {
|
||||
db.data.Clear()
|
||||
db.ttlMap.Clear()
|
||||
db.locker = lock.Make(lockerSize)
|
||||
}
|
||||
|
||||
func (db *DB) Load(db2 *DB) {
|
||||
db.stopWorld.Add(1)
|
||||
defer db.stopWorld.Done()
|
||||
|
||||
db.data = db2.data
|
||||
db.ttlMap = db2.ttlMap
|
||||
db.locker = lock.Make(lockerSize)
|
||||
}
|
||||
|
||||
/* ---- Lock Function ----- */
|
||||
@@ -280,6 +288,7 @@ func (db *DB) Persist(key string) {
|
||||
|
||||
// IsExpired check whether a key is expired
|
||||
func (db *DB) IsExpired(key string) bool {
|
||||
db.stopWorld.Wait()
|
||||
rawExpireTime, ok := db.ttlMap.Get(key)
|
||||
if !ok {
|
||||
return false
|
||||
@@ -295,6 +304,7 @@ func (db *DB) IsExpired(key string) bool {
|
||||
/* --- add version --- */
|
||||
|
||||
func (db *DB) addVersion(keys ...string) {
|
||||
db.stopWorld.Wait()
|
||||
for _, key := range keys {
|
||||
versionCode := db.GetVersion(key)
|
||||
db.versionMap.Put(key, versionCode+1)
|
||||
@@ -303,6 +313,7 @@ func (db *DB) addVersion(keys ...string) {
|
||||
|
||||
// GetVersion returns version code for given key
|
||||
func (db *DB) GetVersion(key string) uint32 {
|
||||
db.stopWorld.Wait()
|
||||
entity, ok := db.versionMap.Get(key)
|
||||
if !ok {
|
||||
return 0
|
||||
@@ -312,6 +323,7 @@ func (db *DB) GetVersion(key string) uint32 {
|
||||
|
||||
// ForEach traverses all the keys in the database
|
||||
func (db *DB) ForEach(cb func(key string, data *database.DataEntity, expiration *time.Time) bool) {
|
||||
db.stopWorld.Wait()
|
||||
db.data.ForEach(func(key string, raw interface{}) bool {
|
||||
entity, _ := raw.(*database.DataEntity)
|
||||
var expiration *time.Time
|
||||
|
@@ -591,18 +591,18 @@ func undoZIncr(db *DB, args [][]byte) []CmdLine {
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("ZAdd", execZAdd, writeFirstKey, undoZAdd, -4)
|
||||
RegisterCommand("ZScore", execZScore, readFirstKey, nil, 3)
|
||||
RegisterCommand("ZIncrBy", execZIncrBy, writeFirstKey, undoZIncr, 4)
|
||||
RegisterCommand("ZRank", execZRank, readFirstKey, nil, 3)
|
||||
RegisterCommand("ZCount", execZCount, readFirstKey, nil, 4)
|
||||
RegisterCommand("ZRevRank", execZRevRank, readFirstKey, nil, 3)
|
||||
RegisterCommand("ZCard", execZCard, readFirstKey, nil, 2)
|
||||
RegisterCommand("ZRange", execZRange, readFirstKey, nil, -4)
|
||||
RegisterCommand("ZRangeByScore", execZRangeByScore, readFirstKey, nil, -4)
|
||||
RegisterCommand("ZRevRange", execZRevRange, readFirstKey, nil, -4)
|
||||
RegisterCommand("ZRevRangeByScore", execZRevRangeByScore, readFirstKey, nil, -4)
|
||||
RegisterCommand("ZRem", execZRem, writeFirstKey, undoZRem, -3)
|
||||
RegisterCommand("ZRemRangeByScore", execZRemRangeByScore, writeFirstKey, rollbackFirstKey, 4)
|
||||
RegisterCommand("ZRemRangeByRank", execZRemRangeByRank, writeFirstKey, rollbackFirstKey, 4)
|
||||
RegisterCommand("ZAdd", execZAdd, writeFirstKey, undoZAdd, -4, flagWrite)
|
||||
RegisterCommand("ZScore", execZScore, readFirstKey, nil, 3, flagReadOnly)
|
||||
RegisterCommand("ZIncrBy", execZIncrBy, writeFirstKey, undoZIncr, 4, flagWrite)
|
||||
RegisterCommand("ZRank", execZRank, readFirstKey, nil, 3, flagReadOnly)
|
||||
RegisterCommand("ZCount", execZCount, readFirstKey, nil, 4, flagReadOnly)
|
||||
RegisterCommand("ZRevRank", execZRevRank, readFirstKey, nil, 3, flagReadOnly)
|
||||
RegisterCommand("ZCard", execZCard, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("ZRange", execZRange, readFirstKey, nil, -4, flagReadOnly)
|
||||
RegisterCommand("ZRangeByScore", execZRangeByScore, readFirstKey, nil, -4, flagReadOnly)
|
||||
RegisterCommand("ZRevRange", execZRevRange, readFirstKey, nil, -4, flagReadOnly)
|
||||
RegisterCommand("ZRevRangeByScore", execZRevRangeByScore, readFirstKey, nil, -4, flagReadOnly)
|
||||
RegisterCommand("ZRem", execZRem, writeFirstKey, undoZRem, -3, flagWrite)
|
||||
RegisterCommand("ZRemRangeByScore", execZRemRangeByScore, writeFirstKey, rollbackFirstKey, 4, flagWrite)
|
||||
RegisterCommand("ZRemRangeByRank", execZRemRangeByRank, writeFirstKey, rollbackFirstKey, 4, flagWrite)
|
||||
}
|
||||
|
@@ -826,29 +826,29 @@ func execBitPos(db *DB, args [][]byte) redis.Reply {
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("Set", execSet, writeFirstKey, rollbackFirstKey, -3)
|
||||
RegisterCommand("SetNx", execSetNX, writeFirstKey, rollbackFirstKey, 3)
|
||||
RegisterCommand("SetEX", execSetEX, writeFirstKey, rollbackFirstKey, 4)
|
||||
RegisterCommand("PSetEX", execPSetEX, writeFirstKey, rollbackFirstKey, 4)
|
||||
RegisterCommand("MSet", execMSet, prepareMSet, undoMSet, -3)
|
||||
RegisterCommand("MGet", execMGet, prepareMGet, nil, -2)
|
||||
RegisterCommand("MSetNX", execMSetNX, prepareMSet, undoMSet, -3)
|
||||
RegisterCommand("Get", execGet, readFirstKey, nil, 2)
|
||||
RegisterCommand("GetEX", execGetEX, writeFirstKey, rollbackFirstKey, -2)
|
||||
RegisterCommand("GetSet", execGetSet, writeFirstKey, rollbackFirstKey, 3)
|
||||
RegisterCommand("GetDel", execGetDel, writeFirstKey, rollbackFirstKey, 2)
|
||||
RegisterCommand("Incr", execIncr, writeFirstKey, rollbackFirstKey, 2)
|
||||
RegisterCommand("IncrBy", execIncrBy, writeFirstKey, rollbackFirstKey, 3)
|
||||
RegisterCommand("IncrByFloat", execIncrByFloat, writeFirstKey, rollbackFirstKey, 3)
|
||||
RegisterCommand("Decr", execDecr, writeFirstKey, rollbackFirstKey, 2)
|
||||
RegisterCommand("DecrBy", execDecrBy, writeFirstKey, rollbackFirstKey, 3)
|
||||
RegisterCommand("StrLen", execStrLen, readFirstKey, nil, 2)
|
||||
RegisterCommand("Append", execAppend, writeFirstKey, rollbackFirstKey, 3)
|
||||
RegisterCommand("SetRange", execSetRange, writeFirstKey, rollbackFirstKey, 4)
|
||||
RegisterCommand("GetRange", execGetRange, readFirstKey, nil, 4)
|
||||
RegisterCommand("SetBit", execSetBit, writeFirstKey, rollbackFirstKey, 4)
|
||||
RegisterCommand("GetBit", execGetBit, readFirstKey, nil, 3)
|
||||
RegisterCommand("BitCount", execBitCount, readFirstKey, nil, -2)
|
||||
RegisterCommand("BitPos", execBitPos, readFirstKey, nil, -3)
|
||||
RegisterCommand("Set", execSet, writeFirstKey, rollbackFirstKey, -3, flagWrite)
|
||||
RegisterCommand("SetNx", execSetNX, writeFirstKey, rollbackFirstKey, 3, flagWrite)
|
||||
RegisterCommand("SetEX", execSetEX, writeFirstKey, rollbackFirstKey, 4, flagWrite)
|
||||
RegisterCommand("PSetEX", execPSetEX, writeFirstKey, rollbackFirstKey, 4, flagWrite)
|
||||
RegisterCommand("MSet", execMSet, prepareMSet, undoMSet, -3, flagWrite)
|
||||
RegisterCommand("MGet", execMGet, prepareMGet, nil, -2, flagReadOnly)
|
||||
RegisterCommand("MSetNX", execMSetNX, prepareMSet, undoMSet, -3, flagWrite)
|
||||
RegisterCommand("Get", execGet, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("GetEX", execGetEX, writeFirstKey, rollbackFirstKey, -2, flagReadOnly)
|
||||
RegisterCommand("GetSet", execGetSet, writeFirstKey, rollbackFirstKey, 3, flagWrite)
|
||||
RegisterCommand("GetDel", execGetDel, writeFirstKey, rollbackFirstKey, 2, flagWrite)
|
||||
RegisterCommand("Incr", execIncr, writeFirstKey, rollbackFirstKey, 2, flagWrite)
|
||||
RegisterCommand("IncrBy", execIncrBy, writeFirstKey, rollbackFirstKey, 3, flagWrite)
|
||||
RegisterCommand("IncrByFloat", execIncrByFloat, writeFirstKey, rollbackFirstKey, 3, flagWrite)
|
||||
RegisterCommand("Decr", execDecr, writeFirstKey, rollbackFirstKey, 2, flagWrite)
|
||||
RegisterCommand("DecrBy", execDecrBy, writeFirstKey, rollbackFirstKey, 3, flagWrite)
|
||||
RegisterCommand("StrLen", execStrLen, readFirstKey, nil, 2, flagReadOnly)
|
||||
RegisterCommand("Append", execAppend, writeFirstKey, rollbackFirstKey, 3, flagWrite)
|
||||
RegisterCommand("SetRange", execSetRange, writeFirstKey, rollbackFirstKey, 4, flagWrite)
|
||||
RegisterCommand("GetRange", execGetRange, readFirstKey, nil, 4, flagReadOnly)
|
||||
RegisterCommand("SetBit", execSetBit, writeFirstKey, rollbackFirstKey, 4, flagWrite)
|
||||
RegisterCommand("GetBit", execGetBit, readFirstKey, nil, 3, flagReadOnly)
|
||||
RegisterCommand("BitCount", execBitCount, readFirstKey, nil, -2, flagReadOnly)
|
||||
RegisterCommand("BitPos", execBitPos, readFirstKey, nil, -3, flagReadOnly)
|
||||
|
||||
}
|
||||
|
@@ -41,5 +41,5 @@ func isAuthenticated(c redis.Connection) bool {
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("ping", Ping, noPrepare, nil, -1)
|
||||
RegisterCommand("ping", Ping, noPrepare, nil, -1, flagReadOnly)
|
||||
}
|
||||
|
@@ -1,17 +1,11 @@
|
||||
package database
|
||||
|
||||
import (
|
||||
"github.com/hdt3213/godis/datastruct/set"
|
||||
"github.com/hdt3213/godis/interface/redis"
|
||||
"github.com/hdt3213/godis/redis/protocol"
|
||||
"strings"
|
||||
)
|
||||
|
||||
var forbiddenInMulti = set.Make(
|
||||
"flushdb",
|
||||
"flushall",
|
||||
)
|
||||
|
||||
// Watch set watching keys
|
||||
func Watch(db *DB, conn redis.Connection, args [][]byte) redis.Reply {
|
||||
watching := conn.GetWatching()
|
||||
@@ -29,7 +23,7 @@ func execGetVersion(db *DB, args [][]byte) redis.Reply {
|
||||
}
|
||||
|
||||
func init() {
|
||||
RegisterCommand("GetVer", execGetVersion, readAllKeys, nil, 2)
|
||||
RegisterCommand("GetVer", execGetVersion, readAllKeys, nil, 2, flagReadOnly)
|
||||
}
|
||||
|
||||
// invoker should lock watching keys
|
||||
@@ -59,9 +53,6 @@ func EnqueueCmd(conn redis.Connection, cmdLine [][]byte) redis.Reply {
|
||||
if !ok {
|
||||
return protocol.MakeErrReply("ERR unknown command '" + cmdName + "'")
|
||||
}
|
||||
if forbiddenInMulti.Has(cmdName) {
|
||||
return protocol.MakeErrReply("ERR command '" + cmdName + "' cannot be used in MULTI")
|
||||
}
|
||||
if cmd.prepare == nil {
|
||||
return protocol.MakeErrReply("ERR command '" + cmdName + "' cannot be used in MULTI")
|
||||
}
|
||||
|
@@ -11,8 +11,6 @@ func makeTestDB() *DB {
|
||||
versionMap: dict.MakeConcurrent(dataDictSize),
|
||||
ttlMap: dict.MakeConcurrent(ttlDictSize),
|
||||
locker: lock.Make(lockerSize),
|
||||
addAof: func(line CmdLine) {
|
||||
|
||||
},
|
||||
addAof: func(line CmdLine) {},
|
||||
}
|
||||
}
|
||||
|
@@ -23,4 +23,7 @@ type Connection interface {
|
||||
// used for multi database
|
||||
GetDBIndex() int
|
||||
SelectDB(int)
|
||||
// returns role of conn, such as connection with client, connection with master node
|
||||
GetRole() int32
|
||||
SetRole(int32)
|
||||
}
|
||||
|
@@ -2,6 +2,6 @@ bind 0.0.0.0
|
||||
port 6399
|
||||
maxclients 128
|
||||
|
||||
appendonly no
|
||||
appendfilename appendonly.aof
|
||||
dbfilename test.rdb
|
||||
#appendonly no
|
||||
#appendfilename appendonly.aof
|
||||
#dbfilename test.rdb
|
||||
|
@@ -8,6 +8,13 @@ import (
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
// NormalCli is client with user
|
||||
NormalCli = iota
|
||||
// ReplicationRecvCli is fake client with replication master
|
||||
ReplicationRecvCli
|
||||
)
|
||||
|
||||
// Connection represents a connection with a redis-cli
|
||||
type Connection struct {
|
||||
conn net.Conn
|
||||
@@ -31,6 +38,7 @@ type Connection struct {
|
||||
|
||||
// selected db
|
||||
selectedDB int
|
||||
role int32
|
||||
}
|
||||
|
||||
// RemoteAddr returns the remote network address
|
||||
@@ -148,6 +156,18 @@ func (c *Connection) ClearQueuedCmds() {
|
||||
c.queue = nil
|
||||
}
|
||||
|
||||
// GetRole returns role of connection, such as connection with master
|
||||
func (c *Connection) GetRole() int32 {
|
||||
if c == nil {
|
||||
return NormalCli
|
||||
}
|
||||
return c.role
|
||||
}
|
||||
|
||||
func (c *Connection) SetRole(r int32) {
|
||||
c.role = r
|
||||
}
|
||||
|
||||
// GetWatching returns watching keys and their version code when started watching
|
||||
func (c *Connection) GetWatching() map[string]uint32 {
|
||||
if c.watching == nil {
|
||||
|
@@ -65,6 +65,7 @@ type readState struct {
|
||||
msgType byte
|
||||
args [][]byte
|
||||
bulkLen int64
|
||||
readingRepl bool
|
||||
}
|
||||
|
||||
func (s *readState) finished() bool {
|
||||
@@ -74,9 +75,10 @@ func (s *readState) finished() bool {
|
||||
func parse0(reader io.Reader, ch chan<- *Payload) {
|
||||
defer func() {
|
||||
if err := recover(); err != nil {
|
||||
logger.Error(string(debug.Stack()))
|
||||
logger.Error(err, string(debug.Stack()))
|
||||
}
|
||||
}()
|
||||
|
||||
bufReader := bufio.NewReader(reader)
|
||||
var state readState
|
||||
var err error
|
||||
@@ -187,16 +189,22 @@ func readLine(bufReader *bufio.Reader, state *readState) ([]byte, bool, error) {
|
||||
return nil, false, errors.New("protocol error: " + string(msg))
|
||||
}
|
||||
} else { // read bulk line (binary safe)
|
||||
msg = make([]byte, state.bulkLen+2)
|
||||
// there is CRLF between BulkReply in normal stream
|
||||
// but there is no CRLF between RDB and following AOF
|
||||
bulkLen := state.bulkLen + 2
|
||||
if state.readingRepl {
|
||||
bulkLen -= 2
|
||||
}
|
||||
msg = make([]byte, bulkLen)
|
||||
_, err = io.ReadFull(bufReader, msg)
|
||||
if err != nil {
|
||||
return nil, true, err
|
||||
}
|
||||
if len(msg) == 0 ||
|
||||
msg[len(msg)-2] != '\r' ||
|
||||
msg[len(msg)-1] != '\n' {
|
||||
return nil, false, errors.New("protocol error: " + string(msg))
|
||||
}
|
||||
//if len(msg) == 0 ||
|
||||
// msg[len(msg)-2] != '\r' ||
|
||||
// msg[len(msg)-1] != '\n' {
|
||||
// return nil, false, errors.New("protocol error: " + string(msg))
|
||||
//}
|
||||
state.bulkLen = 0
|
||||
}
|
||||
return msg, false, nil
|
||||
|
@@ -108,6 +108,11 @@ func (r *StatusReply) ToBytes() []byte {
|
||||
return []byte("+" + r.Status + CRLF)
|
||||
}
|
||||
|
||||
// IsOKReply returns true if the given protocol is +OK
|
||||
func IsOKReply(reply redis.Reply) bool {
|
||||
return string(reply.ToBytes()) == "+OK\r\n"
|
||||
}
|
||||
|
||||
/* ---- Int Reply ---- */
|
||||
|
||||
// IntReply stores an int64 number
|
||||
|
Reference in New Issue
Block a user