replication slave side

This commit is contained in:
hdt3213
2022-05-31 14:15:29 +08:00
committed by finley
parent f327000d3a
commit 380d936ed5
23 changed files with 774 additions and 142 deletions

View File

@@ -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"`

View File

@@ -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)
}

View File

@@ -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()
}

View File

@@ -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)
}

View File

@@ -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)
}

View File

@@ -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)
}

View File

@@ -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)
}

View File

@@ -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
View 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
}

View 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")
}
}

View File

@@ -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
}

View File

@@ -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)
}

View File

@@ -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

View File

@@ -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)
}

View File

@@ -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)
}

View File

@@ -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)
}

View File

@@ -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")
}

View File

@@ -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) {},
}
}

View File

@@ -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)
}

View File

@@ -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

View File

@@ -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 {

View File

@@ -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

View File

@@ -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