raft: make raft configurable

This commit is contained in:
Xiang Li
2015-03-21 18:15:58 -07:00
parent a552722f03
commit d9b5b56c82
9 changed files with 205 additions and 93 deletions

View File

@ -51,6 +51,75 @@ func (st StateType) String() string {
return stmap[uint64(st)]
}
// Config contains the parameters to start a raft.
type Config struct {
// ID is the identity of the local raft. ID cannot be 0.
ID uint64
// Peers contains the IDs of all nodes (including self) in
// the raft cluster. It should only be set when starting a new
// raft cluster.
// Restarting raft from previous configuration will panic if
// Peers is set.
Peers []uint64
// ElectionTick is the election timeout. If a follower does not
// receive any message from the leader of current term during
// ElectionTick, it will become candidate and start an election.
// ElectionTick must be greater than HeartbeatTick. We suggest
// to use ElectionTick = 10 * HeartbeatTick to avoid unnecessary
// leader switching.
ElectionTick int
// HeartbeatTick is the heartbeat interval. A leader sends heartbeat
// message to maintain the leadership every heartbeat interval.
HeartbeatTick int
// Storage is the storage for raft. raft generates entires and
// states to be stored in storage. raft reads the persisted entires
// and states out of Storage when it needs. raft reads out the previous
// state and configuration out of storage when restarting.
Storage Storage
// Applied is the last applied index. It should only be set when restarting
// raft. raft will not return entries to the application smaller or equal to Applied.
// If Applied is unset when restarting, raft might return previous applied entries.
// This is a very application dependent configuration.
Applied uint64
// MaxSizePerMsg limits the max size of each append message. Smaller value lowers
// the raft recovery cost(initial probing and message lost during normal operation).
// On the other side, it might affect the throughput during normal replication.
// Note: math.MaxUint64 for unlimited, 0 for at most one entry per message.
MaxSizePerMsg uint64
// MaxInflightMsgs limits the max number of in-flight append messages during optimistic
// replication phase. The application transportation layer usually has its own sending
// buffer over TCP/UDP. Setting MaxInflightMsgs to avoid overflowing that sending buffer.
// TODO (xiangli): feedback to application to limit the proposal rate?
MaxInflightMsgs int
}
func (c *Config) validate() error {
if c.ID == None {
return errors.New("cannot use none as id")
}
if c.HeartbeatTick <= 0 {
return errors.New("heartbeat tick must be greater than 0")
}
if c.ElectionTick <= c.HeartbeatTick {
return errors.New("election tick must be greater than heartbeat tick")
}
if c.Storage == nil {
return errors.New("storage cannot be nil")
}
if c.MaxInflightMsgs <= 0 {
return errors.New("max inflight messages must be greater than 0")
}
return nil
}
type raft struct {
pb.HardState
@ -83,16 +152,16 @@ type raft struct {
step stepFunc
}
func newRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage,
applied uint64) *raft {
if id == None {
panic("cannot use none id")
func newRaft(c *Config) *raft {
if err := c.validate(); err != nil {
panic(err.Error())
}
raftlog := newLog(storage)
hs, cs, err := storage.InitialState()
raftlog := newLog(c.Storage)
hs, cs, err := c.Storage.InitialState()
if err != nil {
panic(err) // TODO(bdarnell)
}
peers := c.Peers
if len(cs.Nodes) > 0 {
if len(peers) > 0 {
// TODO(bdarnell): the peers argument is always nil except in
@ -103,27 +172,27 @@ func newRaft(id uint64, peers []uint64, election, heartbeat int, storage Storage
peers = cs.Nodes
}
r := &raft{
id: id,
id: c.ID,
lead: None,
raftLog: raftlog,
// 4MB for now and hard code it
// TODO(xiang): add a config arguement into newRaft after we add
// the max inflight message field.
maxMsgSize: 4 * 1024 * 1024,
maxInflight: 256,
maxMsgSize: c.MaxSizePerMsg,
maxInflight: c.MaxInflightMsgs,
prs: make(map[uint64]*Progress),
electionTimeout: election,
heartbeatTimeout: heartbeat,
electionTimeout: c.ElectionTick,
heartbeatTimeout: c.HeartbeatTick,
}
r.rand = rand.New(rand.NewSource(int64(id)))
r.rand = rand.New(rand.NewSource(int64(c.ID)))
for _, p := range peers {
r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
}
if !isHardStateEqual(hs, emptyState) {
r.loadState(hs)
}
if applied > 0 {
raftlog.appliedTo(applied)
if c.Applied > 0 {
raftlog.appliedTo(c.Applied)
}
r.becomeFollower(r.Term, None)