Level sets dependency graph to consume etcd 3.1.5

This commit is contained in:
Timothy St. Clair
2017-04-04 20:54:55 -05:00
parent 1c34102d5b
commit 93c051e28f
392 changed files with 39050 additions and 21582 deletions

View File

@@ -5,15 +5,51 @@ The state machine is kept in sync through the use of a replicated log.
For more details on Raft, see "In Search of an Understandable Consensus Algorithm"
(https://ramcloud.stanford.edu/raft.pdf) by Diego Ongaro and John Ousterhout.
This Raft library is stable and feature complete. As of 2016, it is **the most widely used** Raft library in production, serving tens of thousands clusters each day. It powers distributed systems such as etcd, Kubernetes, Docker Swarm, Cloud Foundry Diego, CockroachDB, TiDB, Project Calico, Flannel, and more.
Most Raft implementations have a monolithic design, including storage handling, messaging serialization, and network transport. This library instead follows a minimalistic design philosophy by only implementing the core raft algorithm. This minimalism buys flexibility, determinism, and performance.
To keep the codebase small as well as provide flexibility, the library only implements the Raft algorithm; both network and disk IO are left to the user. Library users must implement their own transportation layer for message passing between Raft peers over the wire. Similarly, users must implement their own storage layer to persist the Raft log and state.
In order to easily test the Raft library, its behavior should be deterministic. To achieve this determinism, the library models Raft as a state machine. The state machine takes a `Message` as input. A message can either be a local timer update or a network message sent from a remote peer. The state machine's output is a 3-tuple `{[]Messages, []LogEntries, NextState}` consisting of an array of `Messages`, `log entries`, and `Raft state changes`. For state machines with the same state, the same state machine input should always generate the same state machine output.
A simple example application, _raftexample_, is also available to help illustrate
how to use this package in practice:
https://github.com/coreos/etcd/tree/master/contrib/raftexample
# Features
This raft implementation is a full feature implementation of Raft protocol. Features includes:
- Leader election
- Log replication
- Log compaction
- Membership changes
- Leadership transfer extension
- Efficient linearizable read-only queries served by both the leader and followers
- leader checks with quorum and bypasses Raft log before processing read-only queries
- followers asks leader to get a safe read index before processing read-only queries
- More efficient lease-based linearizable read-only queries served by both the leader and followers
- leader bypasses Raft log and processing read-only queries locally
- followers asks leader to get a safe read index before processing read-only queries
- this approach relies on the clock of the all the machines in raft group
This raft implementation also includes a few optional enhancements:
- Optimistic pipelining to reduce log replication latency
- Flow control for log replication
- Batching Raft messages to reduce synchronized network I/O calls
- Batching log entries to reduce disk synchronized I/O
- Writing to leader's disk in parallel
- Internal proposal redirection from followers to leader
- Automatic stepping down when the leader loses quorum
## Notable Users
- [cockroachdb](https://github.com/cockroachdb/cockroach) A Scalable, Survivable, Strongly-Consistent SQL Database
- [dgraph](https://github.com/dgraph-io/dgraph) A Scalable, Distributed, Low Latency, High Throughput Graph Database
- [etcd](https://github.com/coreos/etcd) A distributed reliable key-value store
- [tikv](https://github.com/pingcap/tikv) Distributed transactional key value database powered by Rust and Raft
- [tikv](https://github.com/pingcap/tikv) A Distributed transactional key value database powered by Rust and Raft
- [swarmkit](https://github.com/docker/swarmkit) A toolkit for orchestrating distributed systems at any scale.
## Usage
@@ -21,8 +57,7 @@ https://github.com/coreos/etcd/tree/master/contrib/raftexample
The primary object in raft is a Node. You either start a Node from scratch
using raft.StartNode or start a Node from some initial state using raft.RestartNode.
To start a node from scratch:
To start a three-node cluster
```go
storage := raft.NewMemoryStorage()
c := &Config{
@@ -33,16 +68,30 @@ To start a node from scratch:
MaxSizePerMsg: 4096,
MaxInflightMsgs: 256,
}
// Set peer list to the other nodes in the cluster.
// Note that they need to be started separately as well.
n := raft.StartNode(c, []raft.Peer{{ID: 0x02}, {ID: 0x03}})
```
To restart a node from previous state:
You can start a single node cluster, like so:
```go
// Create storage and config as shown above.
// Set peer list to itself, so this node can become the leader of this single-node cluster.
peers := []raft.Peer{{ID: 0x01}}
n := raft.StartNode(c, peers)
```
To allow a new node to join this cluster, do not pass in any peers. First, you need add the node to the existing cluster by calling `ProposeConfChange` on any existing node inside the cluster. Then, you can start the node with empty peer list, like so:
```go
// Create storage and config as shown above.
n := raft.StartNode(c, nil)
```
To restart a node from previous state:
```go
storage := raft.NewMemoryStorage()
// recover the in-memory storage from persistent
// snapshot, state and entries.
// Recover the in-memory storage from persistent snapshot, state and entries.
storage.ApplySnapshot(snapshot)
storage.SetHardState(state)
storage.Append(entries)
@@ -56,8 +105,8 @@ To restart a node from previous state:
MaxInflightMsgs: 256,
}
// restart raft without peer information.
// peer information is already included in the storage.
// Restart raft without peer information.
// Peer information is already included in the storage.
n := raft.RestartNode(c)
```

View File

@@ -257,6 +257,12 @@ stale log entries:
If candidate receives majority of votes of denials, it reverts back to
follower.
'MsgPreVote' and 'MsgPreVoteResp' are used in an optional two-phase election
protocol. When Config.PreVote is true, a pre-election is carried out first
(using the same rules as a regular election), and no node increases its term
number unless the pre-election indicates that the campaigining node would win.
This minimizes disruption when a partitioned node rejoins the cluster.
'MsgSnap' requests to install a snapshot message. When a node has just
become a leader or the leader receives 'MsgProp' message, it calls
'bcastAppend' method, which then calls 'sendAppend' method to each

View File

@@ -74,8 +74,8 @@ func (l *raftLog) String() string {
// maybeAppend returns (0, false) if the entries cannot be appended. Otherwise,
// it returns (last index of new entries, true).
func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry) (lastnewi uint64, ok bool) {
lastnewi = index + uint64(len(ents))
if l.matchTerm(index, logTerm) {
lastnewi = index + uint64(len(ents))
ci := l.findConflict(ents)
switch {
case ci == 0:
@@ -232,7 +232,7 @@ func (l *raftLog) term(i uint64) (uint64, error) {
if err == nil {
return t, nil
}
if err == ErrCompacted {
if err == ErrCompacted || err == ErrUnavailable {
return 0, err
}
panic(err) // TODO(bdarnell)
@@ -339,7 +339,7 @@ func (l *raftLog) mustCheckOutOfBounds(lo, hi uint64) error {
return ErrCompacted
}
length := l.lastIndex() - fi + 1
length := l.lastIndex() + 1 - fi
if lo < fi || hi > fi+length {
l.logger.Panicf("slice[%d,%d) out of bound [%d,%d]", lo, hi, fi, l.lastIndex())
}

View File

@@ -101,23 +101,23 @@ func (u *unstable) restore(s pb.Snapshot) {
}
func (u *unstable) truncateAndAppend(ents []pb.Entry) {
after := ents[0].Index - 1
after := ents[0].Index
switch {
case after == u.offset+uint64(len(u.entries))-1:
// after is the last index in the u.entries
case after == u.offset+uint64(len(u.entries)):
// after is the next index in the u.entries
// directly append
u.entries = append(u.entries, ents...)
case after < u.offset:
u.logger.Infof("replace the unstable entries from index %d", after+1)
case after <= u.offset:
u.logger.Infof("replace the unstable entries from index %d", after)
// The log is being truncated to before our current offset
// portion, so set the offset and replace the entries
u.offset = after + 1
u.offset = after
u.entries = ents
default:
// truncate to after and copy to u.entries
// then append
u.logger.Infof("truncate the unstable entries to index %d", after)
u.entries = append([]pb.Entry{}, u.slice(u.offset, after+1)...)
u.logger.Infof("truncate the unstable entries before index %d", after)
u.entries = append([]pb.Entry{}, u.slice(u.offset, after)...)
u.entries = append(u.entries, ents...)
}
}

View File

@@ -60,6 +60,12 @@ type Ready struct {
// HardState will be equal to empty state if there is no update.
pb.HardState
// ReadStates can be used for node to serve linearizable read requests locally
// when its applied index is greater than the index in ReadState.
// Note that the readState will be returned when raft receives msgReadIndex.
// The returned is only valid for the request that requested to read.
ReadStates []ReadState
// Entries specifies entries to be saved to stable storage BEFORE
// Messages are sent.
Entries []pb.Entry
@@ -96,7 +102,7 @@ func IsEmptySnap(sp pb.Snapshot) bool {
func (rd Ready) containsUpdates() bool {
return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) ||
!IsEmptySnap(rd.Snapshot) || len(rd.Entries) > 0 ||
len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0
len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0 || len(rd.ReadStates) != 0
}
// Node represents a node in a raft cluster.
@@ -130,27 +136,23 @@ type Node interface {
// However, as an optimization, the application may call Advance while it is applying the
// commands. For example. when the last Ready contains a snapshot, the application might take
// a long time to apply the snapshot data. To continue receiving Ready without blocking raft
// progress, it can call Advance before finish applying the last ready. To make this optimization
// work safely, when the application receives a Ready with softState.RaftState equal to Candidate
// it MUST apply all pending configuration changes if there is any.
//
// Here is a simple solution that waiting for ALL pending entries to get applied.
// ```
// ...
// rd := <-n.Ready()
// go apply(rd.CommittedEntries) // optimization to apply asynchronously in FIFO order.
// if rd.SoftState.RaftState == StateCandidate {
// waitAllApplied()
// }
// n.Advance()
// ...
//```
// progress, it can call Advance before finishing applying the last ready.
Advance()
// ApplyConfChange applies config change to the local node.
// Returns an opaque ConfState protobuf which must be recorded
// in snapshots. Will never return nil; it returns a pointer only
// to match MemoryStorage.Compact.
ApplyConfChange(cc pb.ConfChange) *pb.ConfState
// TransferLeadership attempts to transfer leadership to the given transferee.
TransferLeadership(ctx context.Context, lead, transferee uint64)
// ReadIndex request a read state. The read state will be set in the ready.
// Read state has a read index. Once the application advances further than the read
// index, any linearizable read requests issued before the read request can be
// processed safely. The read state will have the same rctx attached.
ReadIndex(ctx context.Context, rctx []byte) error
// Status returns the current status of the raft state machine.
Status() Status
// ReportUnreachable reports the given node is not reachable for the last send.
@@ -361,7 +363,9 @@ func (n *node) run(r *raft) {
if !IsEmptySnap(rd.Snapshot) {
prevSnapi = rd.Snapshot.Metadata.Index
}
r.msgs = nil
r.readStates = nil
advancec = n.advancec
case <-advancec:
if prevHardSt.Commit != 0 {
@@ -458,8 +462,12 @@ func (n *node) ApplyConfChange(cc pb.ConfChange) *pb.ConfState {
func (n *node) Status() Status {
c := make(chan Status)
n.status <- c
return <-c
select {
case n.status <- c:
return <-c
case <-n.done:
return Status{}
}
}
func (n *node) ReportUnreachable(id uint64) {
@@ -478,6 +486,19 @@ func (n *node) ReportSnapshot(id uint64, status SnapshotStatus) {
}
}
func (n *node) TransferLeadership(ctx context.Context, lead, transferee uint64) {
select {
// manually set 'from' and 'to', so that leader can voluntarily transfers its leadership
case n.recvc <- pb.Message{Type: pb.MsgTransferLeader, From: transferee, To: lead}:
case <-n.done:
case <-ctx.Done():
}
}
func (n *node) ReadIndex(ctx context.Context, rctx []byte) error {
return n.step(ctx, pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}})
}
func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready {
rd := Ready{
Entries: r.raftLog.unstableEntries(),
@@ -493,5 +514,8 @@ func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready {
if r.raftLog.unstable.snapshot != nil {
rd.Snapshot = *r.raftLog.unstable.snapshot
}
if len(r.readStates) != 0 {
rd.ReadStates = r.readStates
}
return rd
}

View File

@@ -64,12 +64,17 @@ type Progress struct {
RecentActive bool
// inflights is a sliding window for the inflight messages.
// Each inflight message contains one or more log entries.
// The max number of entries per message is defined in raft config as MaxSizePerMsg.
// Thus inflight effectively limits both the number of inflight messages
// and the bandwidth each Progress can use.
// When inflights is full, no more message should be sent.
// When a leader sends out a message, the index of the last
// entry should be added to inflights. The index MUST be added
// into inflights in order.
// When a leader receives a reply, the previous inflights should
// be freed by calling inflights.freeTo.
// be freed by calling inflights.freeTo with the index of the last
// received entry.
ins *inflights
}
@@ -150,8 +155,11 @@ func (pr *Progress) maybeDecrTo(rejected, last uint64) bool {
func (pr *Progress) pause() { pr.Paused = true }
func (pr *Progress) resume() { pr.Paused = false }
// isPaused returns whether progress stops sending message.
func (pr *Progress) isPaused() bool {
// IsPaused returns whether sending log entries to this node has been
// paused. A node may be paused because it has rejected recent
// MsgApps, is currently waiting for a snapshot, or has reached the
// MaxInflightMsgs limit.
func (pr *Progress) IsPaused() bool {
switch pr.State {
case ProgressStateProbe:
return pr.Paused
@@ -173,7 +181,7 @@ func (pr *Progress) needSnapshotAbort() bool {
}
func (pr *Progress) String() string {
return fmt.Sprintf("next = %d, match = %d, state = %s, waiting = %v, pendingSnapshot = %d", pr.Next, pr.Match, pr.State, pr.isPaused(), pr.PendingSnapshot)
return fmt.Sprintf("next = %d, match = %d, state = %s, waiting = %v, pendingSnapshot = %d", pr.Next, pr.Match, pr.State, pr.IsPaused(), pr.PendingSnapshot)
}
type inflights struct {
@@ -183,14 +191,16 @@ type inflights struct {
count int
// the size of the buffer
size int
size int
// buffer contains the index of the last entry
// inside one message.
buffer []uint64
}
func newInflights(size int) *inflights {
return &inflights{
size: size,
buffer: make([]uint64, size),
size: size,
}
}
@@ -200,13 +210,32 @@ func (in *inflights) add(inflight uint64) {
panic("cannot add into a full inflights")
}
next := in.start + in.count
if next >= in.size {
next -= in.size
size := in.size
if next >= size {
next -= size
}
if next >= len(in.buffer) {
in.growBuf()
}
in.buffer[next] = inflight
in.count++
}
// grow the inflight buffer by doubling up to inflights.size. We grow on demand
// instead of preallocating to inflights.size to handle systems which have
// thousands of Raft groups per process.
func (in *inflights) growBuf() {
newSize := len(in.buffer) * 2
if newSize == 0 {
newSize = 1
} else if newSize > in.size {
newSize = in.size
}
newBuffer := make([]uint64, newSize)
copy(newBuffer, in.buffer)
in.buffer = newBuffer
}
// freeTo frees the inflights smaller or equal to the given `to` flight.
func (in *inflights) freeTo(to uint64) {
if in.count == 0 || to < in.buffer[in.start] {
@@ -221,13 +250,19 @@ func (in *inflights) freeTo(to uint64) {
}
// increase index and maybe rotate
if idx++; idx >= in.size {
idx -= in.size
size := in.size
if idx++; idx >= size {
idx -= size
}
}
// free i inflights and set new start index
in.count -= i
in.start = idx
if in.count == 0 {
// inflights is empty, reset the start index so that we don't grow the
// buffer unnecessarily.
in.start = 0
}
}
func (in *inflights) freeFirstOne() { in.freeTo(in.buffer[in.start]) }

View File

@@ -15,12 +15,15 @@
package raft
import (
"bytes"
"errors"
"fmt"
"math"
"math/rand"
"sort"
"strings"
"sync"
"time"
pb "github.com/coreos/etcd/raft/raftpb"
)
@@ -34,8 +37,60 @@ const (
StateFollower StateType = iota
StateCandidate
StateLeader
StatePreCandidate
numStates
)
type ReadOnlyOption int
const (
// ReadOnlySafe guarantees the linearizability of the read only request by
// communicating with the quorum. It is the default and suggested option.
ReadOnlySafe ReadOnlyOption = iota
// ReadOnlyLeaseBased ensures linearizability of the read only request by
// relying on the leader lease. It can be affected by clock drift.
// If the clock drift is unbounded, leader might keep the lease longer than it
// should (clock can move backward/pause without any bound). ReadIndex is not safe
// in that case.
ReadOnlyLeaseBased
)
// Possible values for CampaignType
const (
// campaignPreElection represents the first phase of a normal election when
// Config.PreVote is true.
campaignPreElection CampaignType = "CampaignPreElection"
// campaignElection represents a normal (time-based) election (the second phase
// of the election when Config.PreVote is true).
campaignElection CampaignType = "CampaignElection"
// campaignTransfer represents the type of leader transfer
campaignTransfer CampaignType = "CampaignTransfer"
)
// lockedRand is a small wrapper around rand.Rand to provide
// synchronization. Only the methods needed by the code are exposed
// (e.g. Intn).
type lockedRand struct {
mu sync.Mutex
rand *rand.Rand
}
func (r *lockedRand) Intn(n int) int {
r.mu.Lock()
v := r.rand.Intn(n)
r.mu.Unlock()
return v
}
var globalRand = &lockedRand{
rand: rand.New(rand.NewSource(time.Now().UnixNano())),
}
// CampaignType represents the type of campaigning
// the reason we use the type of string instead of uint64
// is because it's simpler to compare and fill in raft entries
type CampaignType string
// StateType represents the role of a node in a cluster.
type StateType uint64
@@ -43,6 +98,7 @@ var stmap = [...]string{
"StateFollower",
"StateCandidate",
"StateLeader",
"StatePreCandidate",
}
func (st StateType) String() string {
@@ -100,6 +156,23 @@ type Config struct {
// steps down when quorum is not active for an electionTimeout.
CheckQuorum bool
// PreVote enables the Pre-Vote algorithm described in raft thesis section
// 9.6. This prevents disruption when a node that has been partitioned away
// rejoins the cluster.
PreVote bool
// ReadOnlyOption specifies how the read only request is processed.
//
// ReadOnlySafe guarantees the linearizability of the read only request by
// communicating with the quorum. It is the default and suggested option.
//
// ReadOnlyLeaseBased ensures linearizability of the read only request by
// relying on the leader lease. It can be affected by clock drift.
// If the clock drift is unbounded, leader might keep the lease longer than it
// should (clock can move backward/pause without any bound). ReadIndex is not safe
// in that case.
ReadOnlyOption ReadOnlyOption
// Logger is the logger used for raft log. For multinode which can host
// multiple raft group, each raft group can have its own logger
Logger Logger
@@ -139,6 +212,8 @@ type raft struct {
Term uint64
Vote uint64
readStates []ReadState
// the log
raftLog *raftLog
@@ -160,6 +235,8 @@ type raft struct {
// New configuration is ignored if there exists unapplied configuration.
pendingConf bool
readOnly *readOnly
// number of ticks since it reached last electionTimeout when it is leader
// or candidate.
// number of ticks since it reached last electionTimeout or received a
@@ -171,6 +248,7 @@ type raft struct {
heartbeatElapsed int
checkQuorum bool
preVote bool
heartbeatTimeout int
electionTimeout int
@@ -179,7 +257,6 @@ type raft struct {
// when raft changes its state to follower or candidate.
randomizedElectionTimeout int
rand *rand.Rand
tick func()
step stepFunc
@@ -216,8 +293,9 @@ func newRaft(c *Config) *raft {
heartbeatTimeout: c.HeartbeatTick,
logger: c.Logger,
checkQuorum: c.CheckQuorum,
preVote: c.PreVote,
readOnly: newReadOnly(c.ReadOnlyOption),
}
r.rand = rand.New(rand.NewSource(int64(c.ID)))
for _, p := range peers {
r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
}
@@ -265,11 +343,23 @@ func (r *raft) nodes() []uint64 {
// send persists state to stable storage and then sends to its mailbox.
func (r *raft) send(m pb.Message) {
m.From = r.id
// do not attach term to MsgProp
// proposals are a way to forward to the leader and
// should be treated as local message.
if m.Type != pb.MsgProp {
m.Term = r.Term
if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
if m.Term == 0 {
// PreVote RPCs are sent at a term other than our actual term, so the code
// that sends these messages is responsible for setting the term.
panic(fmt.Sprintf("term should be set when sending %s", m.Type))
}
} else {
if m.Term != 0 {
panic(fmt.Sprintf("term should not be set when sending %s (was %d)", m.Type, m.Term))
}
// do not attach term to MsgProp, MsgReadIndex
// proposals are a way to forward to the leader and
// should be treated as local message.
// MsgReadIndex is also forwarded to leader.
if m.Type != pb.MsgProp && m.Type != pb.MsgReadIndex {
m.Term = r.Term
}
}
r.msgs = append(r.msgs, m)
}
@@ -277,7 +367,7 @@ func (r *raft) send(m pb.Message) {
// sendAppend sends RPC, with entries to the given peer.
func (r *raft) sendAppend(to uint64) {
pr := r.prs[to]
if pr.isPaused() {
if pr.IsPaused() {
return
}
m := pb.Message{}
@@ -334,7 +424,7 @@ func (r *raft) sendAppend(to uint64) {
}
// sendHeartbeat sends an empty MsgApp
func (r *raft) sendHeartbeat(to uint64) {
func (r *raft) sendHeartbeat(to uint64, ctx []byte) {
// Attach the commit as min(to.matched, r.committed).
// When the leader sends out heartbeat message,
// the receiver(follower) might not be matched with the leader
@@ -343,10 +433,12 @@ func (r *raft) sendHeartbeat(to uint64) {
// an unmatched index.
commit := min(r.prs[to].Match, r.raftLog.committed)
m := pb.Message{
To: to,
Type: pb.MsgHeartbeat,
Commit: commit,
To: to,
Type: pb.MsgHeartbeat,
Commit: commit,
Context: ctx,
}
r.send(m)
}
@@ -363,12 +455,20 @@ func (r *raft) bcastAppend() {
// bcastHeartbeat sends RPC, without entries to all the peers.
func (r *raft) bcastHeartbeat() {
lastCtx := r.readOnly.lastPendingRequestCtx()
if len(lastCtx) == 0 {
r.bcastHeartbeatWithCtx(nil)
} else {
r.bcastHeartbeatWithCtx([]byte(lastCtx))
}
}
func (r *raft) bcastHeartbeatWithCtx(ctx []byte) {
for id := range r.prs {
if id == r.id {
continue
}
r.sendHeartbeat(id)
r.prs[id].resume()
r.sendHeartbeat(id, ctx)
}
}
@@ -407,6 +507,7 @@ func (r *raft) reset(term uint64) {
}
}
r.pendingConf = false
r.readOnly = newReadOnly(r.readOnly.option)
}
func (r *raft) appendEntry(es ...pb.Entry) {
@@ -479,6 +580,20 @@ func (r *raft) becomeCandidate() {
r.logger.Infof("%x became candidate at term %d", r.id, r.Term)
}
func (r *raft) becomePreCandidate() {
// TODO(xiangli) remove the panic when the raft implementation is stable
if r.state == StateLeader {
panic("invalid transition [leader -> pre-candidate]")
}
// Becoming a pre-candidate changes our step functions and state,
// but doesn't change anything else. In particular it does not increase
// r.Term or change r.Vote.
r.step = stepCandidate
r.tick = r.tickElection
r.state = StatePreCandidate
r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term)
}
func (r *raft) becomeLeader() {
// TODO(xiangli) remove the panic when the raft implementation is stable
if r.state == StateFollower {
@@ -494,40 +609,61 @@ func (r *raft) becomeLeader() {
r.logger.Panicf("unexpected error getting uncommitted entries (%v)", err)
}
for _, e := range ents {
if e.Type != pb.EntryConfChange {
continue
}
if r.pendingConf {
panic("unexpected double uncommitted config entry")
}
nconf := numOfPendingConf(ents)
if nconf > 1 {
panic("unexpected multiple uncommitted config entry")
}
if nconf == 1 {
r.pendingConf = true
}
r.appendEntry(pb.Entry{Data: nil})
r.logger.Infof("%x became leader at term %d", r.id, r.Term)
}
func (r *raft) campaign() {
r.becomeCandidate()
if r.quorum() == r.poll(r.id, true) {
r.becomeLeader()
func (r *raft) campaign(t CampaignType) {
var term uint64
var voteMsg pb.MessageType
if t == campaignPreElection {
r.becomePreCandidate()
voteMsg = pb.MsgPreVote
// PreVote RPCs are sent for the next term before we've incremented r.Term.
term = r.Term + 1
} else {
r.becomeCandidate()
voteMsg = pb.MsgVote
term = r.Term
}
if r.quorum() == r.poll(r.id, voteRespMsgType(voteMsg), true) {
// We won the election after voting for ourselves (which must mean that
// this is a single-node cluster). Advance to the next state.
if t == campaignPreElection {
r.campaign(campaignElection)
} else {
r.becomeLeader()
}
return
}
for id := range r.prs {
if id == r.id {
continue
}
r.logger.Infof("%x [logterm: %d, index: %d] sent vote request to %x at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), id, r.Term)
r.send(pb.Message{To: id, Type: pb.MsgVote, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm()})
r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), voteMsg, id, r.Term)
var ctx []byte
if t == campaignTransfer {
ctx = []byte(t)
}
r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx})
}
}
func (r *raft) poll(id uint64, v bool) (granted int) {
func (r *raft) poll(id uint64, t pb.MessageType, v bool) (granted int) {
if v {
r.logger.Infof("%x received vote from %x at term %d", r.id, id, r.Term)
r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term)
} else {
r.logger.Infof("%x received vote rejection from %x at term %d", r.id, id, r.Term)
r.logger.Infof("%x received %s rejection from %x at term %d", r.id, t, id, r.Term)
}
if _, ok := r.votes[id]; !ok {
r.votes[id] = v
@@ -541,50 +677,54 @@ func (r *raft) poll(id uint64, v bool) (granted int) {
}
func (r *raft) Step(m pb.Message) error {
if m.Type == pb.MsgHup {
if r.state != StateLeader {
r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
r.campaign()
} else {
r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
}
return nil
}
if m.Type == pb.MsgTransferLeader {
if r.state != StateLeader {
r.logger.Debugf("%x [term %d state %v] ignoring MsgTransferLeader to %x", r.id, r.Term, r.state, m.From)
}
}
// Handle the message term, which may result in our stepping down to a follower.
switch {
case m.Term == 0:
// local message
case m.Term > r.Term:
lead := m.From
if m.Type == pb.MsgVote {
if r.checkQuorum && r.state != StateCandidate && r.electionElapsed < r.electionTimeout {
if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
force := bytes.Equal(m.Context, []byte(campaignTransfer))
inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout
if !force && inLease {
// If a server receives a RequestVote request within the minimum election timeout
// of hearing from a current leader, it does not update its term or grant its vote
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored vote from %x [logterm: %d, index: %d] at term %d: lease is not expired (remaining ticks: %d)",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored %s from %x [logterm: %d, index: %d] at term %d: lease is not expired (remaining ticks: %d)",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
return nil
}
lead = None
}
r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
r.id, r.Term, m.Type, m.From, m.Term)
r.becomeFollower(m.Term, lead)
switch {
case m.Type == pb.MsgPreVote:
// Never change our term in response to a PreVote
case m.Type == pb.MsgPreVoteResp && !m.Reject:
// We send pre-vote requests with a term in our future. If the
// pre-vote is granted, we will increment our term when we get a
// quorum. If it is not, the term comes from the node that
// rejected our vote so we should become a follower at the new
// term.
default:
r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
r.id, r.Term, m.Type, m.From, m.Term)
r.becomeFollower(m.Term, lead)
}
case m.Term < r.Term:
if r.checkQuorum && (m.Type == pb.MsgHeartbeat || m.Type == pb.MsgApp) {
// We have received messages from a leader at a lower term. It is possible that these messages were
// simply delayed in the network, but this could also mean that this node has advanced its term number
// during a network partition, and it is now unable to either win an election or to rejoin the majority
// on the old term. If checkQuorum is false, this will be handled by incrementing term numbers in response
// to MsgVote with a higher term, but if checkQuorum is true we may not advance the term on MsgVote and
// must generate other messages to advance the term. The net result of these two features is to minimize
// the disruption caused by nodes that have been removed from the cluster's configuration: a removed node
// will send MsgVotes which will be ignored, but it will not receive MsgApp or MsgHeartbeat, so it will not
// create disruptive term increases
// We have received messages from a leader at a lower term. It is possible
// that these messages were simply delayed in the network, but this could
// also mean that this node has advanced its term number during a network
// partition, and it is now unable to either win an election or to rejoin
// the majority on the old term. If checkQuorum is false, this will be
// handled by incrementing term numbers in response to MsgVote with a
// higher term, but if checkQuorum is true we may not advance the term on
// MsgVote and must generate other messages to advance the term. The net
// result of these two features is to minimize the disruption caused by
// nodes that have been removed from the cluster's configuration: a
// removed node will send MsgVotes (or MsgPreVotes) which will be ignored,
// but it will not receive MsgApp or MsgHeartbeat, so it will not create
// disruptive term increases
r.send(pb.Message{To: m.From, Type: pb.MsgAppResp})
} else {
// ignore other cases
@@ -593,7 +733,50 @@ func (r *raft) Step(m pb.Message) error {
}
return nil
}
r.step(r, m)
switch m.Type {
case pb.MsgHup:
if r.state != StateLeader {
ents, err := r.raftLog.slice(r.raftLog.applied+1, r.raftLog.committed+1, noLimit)
if err != nil {
r.logger.Panicf("unexpected error getting unapplied entries (%v)", err)
}
if n := numOfPendingConf(ents); n != 0 && r.raftLog.committed > r.raftLog.applied {
r.logger.Warningf("%x cannot campaign at term %d since there are still %d pending configuration changes to apply", r.id, r.Term, n)
return nil
}
r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
if r.preVote {
r.campaign(campaignPreElection)
} else {
r.campaign(campaignElection)
}
} else {
r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
}
case pb.MsgVote, pb.MsgPreVote:
// The m.Term > r.Term clause is for MsgPreVote. For MsgVote m.Term should
// always equal r.Term.
if (r.Vote == None || m.Term > r.Term || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] cast %s for %x [logterm: %d, index: %d] at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type)})
if m.Type == pb.MsgVote {
// Only record real votes.
r.electionElapsed = 0
r.Vote = m.From
}
} else {
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
r.send(pb.Message{To: m.From, Type: voteRespMsgType(m.Type), Reject: true})
}
default:
r.step(r, m)
}
return nil
}
@@ -629,6 +812,7 @@ func stepLeader(r *raft, m pb.Message) {
for i, e := range m.Entries {
if e.Type == pb.EntryConfChange {
if r.pendingConf {
r.logger.Infof("propose conf %s ignored since pending unapplied configuration", e.String())
m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
}
r.pendingConf = true
@@ -637,10 +821,35 @@ func stepLeader(r *raft, m pb.Message) {
r.appendEntry(m.Entries...)
r.bcastAppend()
return
case pb.MsgVote:
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
case pb.MsgReadIndex:
if r.quorum() > 1 {
if r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(r.raftLog.committed)) != r.Term {
// Reject read only request when this leader has not committed any log entry at its term.
return
}
// thinking: use an interally defined context instead of the user given context.
// We can express this in terms of the term and index instead of a user-supplied value.
// This would allow multiple reads to piggyback on the same message.
switch r.readOnly.option {
case ReadOnlySafe:
r.readOnly.addRequest(r.raftLog.committed, m)
r.bcastHeartbeatWithCtx(m.Entries[0].Data)
case ReadOnlyLeaseBased:
var ri uint64
if r.checkQuorum {
ri = r.raftLog.committed
}
if m.From == None || m.From == r.id { // from local member
r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
} else {
r.send(pb.Message{To: m.From, Type: pb.MsgReadIndexResp, Index: ri, Entries: m.Entries})
}
}
} else {
r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
}
return
}
@@ -665,7 +874,7 @@ func stepLeader(r *raft, m pb.Message) {
r.sendAppend(m.From)
}
} else {
oldPaused := pr.isPaused()
oldPaused := pr.IsPaused()
if pr.maybeUpdate(m.Index) {
switch {
case pr.State == ProgressStateProbe:
@@ -693,6 +902,7 @@ func stepLeader(r *raft, m pb.Message) {
}
case pb.MsgHeartbeatResp:
pr.RecentActive = true
pr.resume()
// free one slot for the full inflights window to allow progress.
if pr.State == ProgressStateReplicate && pr.ins.full() {
@@ -701,6 +911,25 @@ func stepLeader(r *raft, m pb.Message) {
if pr.Match < r.raftLog.lastIndex() {
r.sendAppend(m.From)
}
if r.readOnly.option != ReadOnlySafe || len(m.Context) == 0 {
return
}
ackCount := r.readOnly.recvAck(m)
if ackCount < r.quorum() {
return
}
rss := r.readOnly.advance(m)
for _, rs := range rss {
req := rs.req
if req.From == None || req.From == r.id { // from local member
r.readStates = append(r.readStates, ReadState{Index: rs.index, RequestCtx: req.Entries[0].Data})
} else {
r.send(pb.Message{To: req.From, Type: pb.MsgReadIndexResp, Index: rs.index, Entries: req.Entries})
}
}
case pb.MsgSnapStatus:
if pr.State != ProgressStateSnapshot {
return
@@ -754,7 +983,18 @@ func stepLeader(r *raft, m pb.Message) {
}
}
// stepCandidate is shared by StateCandidate and StatePreCandidate; the difference is
// whether they respond to MsgVoteResp or MsgPreVoteResp.
func stepCandidate(r *raft, m pb.Message) {
// Only handle vote responses corresponding to our candidacy (while in
// StateCandidate, we may get stale MsgPreVoteResp messages in this term from
// our pre-candidate state).
var myVoteRespType pb.MessageType
if r.state == StatePreCandidate {
myVoteRespType = pb.MsgPreVoteResp
} else {
myVoteRespType = pb.MsgVoteResp
}
switch m.Type {
case pb.MsgProp:
r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
@@ -768,17 +1008,17 @@ func stepCandidate(r *raft, m pb.Message) {
case pb.MsgSnap:
r.becomeFollower(m.Term, m.From)
r.handleSnapshot(m)
case pb.MsgVote:
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
case pb.MsgVoteResp:
gr := r.poll(m.From, !m.Reject)
r.logger.Infof("%x [quorum:%d] has received %d votes and %d vote rejections", r.id, r.quorum(), gr, len(r.votes)-gr)
case myVoteRespType:
gr := r.poll(m.From, m.Type, !m.Reject)
r.logger.Infof("%x [quorum:%d] has received %d %s votes and %d vote rejections", r.id, r.quorum(), gr, m.Type, len(r.votes)-gr)
switch r.quorum() {
case gr:
r.becomeLeader()
r.bcastAppend()
if r.state == StatePreCandidate {
r.campaign(campaignElection)
} else {
r.becomeLeader()
r.bcastAppend()
}
case len(r.votes) - gr:
r.becomeFollower(r.Term, None)
}
@@ -806,22 +1046,38 @@ func stepFollower(r *raft, m pb.Message) {
r.handleHeartbeat(m)
case pb.MsgSnap:
r.electionElapsed = 0
r.lead = m.From
r.handleSnapshot(m)
case pb.MsgVote:
if (r.Vote == None || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
r.electionElapsed = 0
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] voted for %x [logterm: %d, index: %d] at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
r.Vote = m.From
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp})
} else {
r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
case pb.MsgTransferLeader:
if r.lead == None {
r.logger.Infof("%x no leader at term %d; dropping leader transfer msg", r.id, r.Term)
return
}
m.To = r.lead
r.send(m)
case pb.MsgTimeoutNow:
r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From)
r.campaign()
if r.promotable() {
r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From)
// Leadership transfers never use pre-vote even if r.preVote is true; we
// know we are not recovering from a partition so there is no need for the
// extra round trip.
r.campaign(campaignTransfer)
} else {
r.logger.Infof("%x received MsgTimeoutNow from %x but is not promotable", r.id, m.From)
}
case pb.MsgReadIndex:
if r.lead == None {
r.logger.Infof("%x no leader at term %d; dropping index reading msg", r.id, r.Term)
return
}
m.To = r.lead
r.send(m)
case pb.MsgReadIndexResp:
if len(m.Entries) != 1 {
r.logger.Errorf("%x invalid format of MsgReadIndexResp from %x, entries count: %d", r.id, m.From, len(m.Entries))
return
}
r.readStates = append(r.readStates, ReadState{Index: m.Index, RequestCtx: m.Entries[0].Data})
}
}
@@ -842,7 +1098,7 @@ func (r *raft) handleAppendEntries(m pb.Message) {
func (r *raft) handleHeartbeat(m pb.Message) {
r.raftLog.commitTo(m.Commit)
r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp})
r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp, Context: m.Context})
}
func (r *raft) handleSnapshot(m pb.Message) {
@@ -877,11 +1133,9 @@ func (r *raft) restore(s pb.Snapshot) bool {
r.raftLog.restore(s)
r.prs = make(map[uint64]*Progress)
for _, n := range s.Metadata.ConfState.Nodes {
match, next := uint64(0), uint64(r.raftLog.lastIndex())+1
match, next := uint64(0), r.raftLog.lastIndex()+1
if n == r.id {
match = next - 1
} else {
match = 0
}
r.setProgress(n, match, next)
r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n])
@@ -897,6 +1151,7 @@ func (r *raft) promotable() bool {
}
func (r *raft) addNode(id uint64) {
r.pendingConf = false
if _, ok := r.prs[id]; ok {
// Ignore any redundant addNode calls (which can happen because the
// initial bootstrapping entries are applied twice).
@@ -904,7 +1159,6 @@ func (r *raft) addNode(id uint64) {
}
r.setProgress(id, 0, r.raftLog.lastIndex()+1)
r.pendingConf = false
}
func (r *raft) removeNode(id uint64) {
@@ -954,7 +1208,7 @@ func (r *raft) pastElectionTimeout() bool {
}
func (r *raft) resetRandomizedElectionTimeout() {
r.randomizedElectionTimeout = r.electionTimeout + r.rand.Intn(r.electionTimeout)
r.randomizedElectionTimeout = r.electionTimeout + globalRand.Intn(r.electionTimeout)
}
// checkQuorumActive returns true if the quorum is active from
@@ -987,3 +1241,13 @@ func (r *raft) sendTimeoutNow(to uint64) {
func (r *raft) abortLeaderTransfer() {
r.leadTransferee = None
}
func numOfPendingConf(ents []pb.Entry) int {
n := 0
for i := range ents {
if ents[i].Type == pb.EntryConfChange {
n++
}
}
return n
}

File diff suppressed because it is too large Load Diff

View File

@@ -48,6 +48,10 @@ enum MessageType {
MsgCheckQuorum = 12;
MsgTransferLeader = 13;
MsgTimeoutNow = 14;
MsgReadIndex = 15;
MsgReadIndexResp = 16;
MsgPreVote = 17;
MsgPreVoteResp = 18;
}
message Message {
@@ -62,6 +66,7 @@ message Message {
optional Snapshot snapshot = 9 [(gogoproto.nullable) = false];
optional bool reject = 10 [(gogoproto.nullable) = false];
optional uint64 rejectHint = 11 [(gogoproto.nullable) = false];
optional bytes context = 12;
}
message HardState {

View File

@@ -66,6 +66,9 @@ func (rn *RawNode) commitReady(rd Ready) {
if !IsEmptySnap(rd.Snapshot) {
rn.raft.raftLog.stableSnapTo(rd.Snapshot.Metadata.Index)
}
if len(rd.ReadStates) != 0 {
rn.raft.readStates = nil
}
}
// NewRawNode returns a new RawNode given configuration and a list of raft peers.
@@ -103,9 +106,14 @@ func NewRawNode(config *Config, peers []Peer) (*RawNode, error) {
r.addNode(peer.ID)
}
}
// Set the initial hard and soft states after performing all initialization.
rn.prevSoftSt = r.softState()
rn.prevHardSt = r.hardState()
if lastIndex == 0 {
rn.prevHardSt = emptyState
} else {
rn.prevHardSt = r.hardState()
}
return rn, nil
}
@@ -115,6 +123,18 @@ func (rn *RawNode) Tick() {
rn.raft.tick()
}
// TickQuiesced advances the internal logical clock by a single tick without
// performing any other state machine processing. It allows the caller to avoid
// periodic heartbeats and elections when all of the peers in a Raft group are
// known to be at the same state. Expected usage is to periodically invoke Tick
// or TickQuiesced depending on whether the group is "active" or "quiesced".
//
// WARNING: Be very careful about using this method as it subverts the Raft
// state machine. You should probably be using Tick instead.
func (rn *RawNode) TickQuiesced() {
rn.raft.electionElapsed++
}
// Campaign causes this RawNode to transition to candidate state.
func (rn *RawNode) Campaign() error {
return rn.raft.Step(pb.Message{
@@ -200,6 +220,9 @@ func (rn *RawNode) HasReady() bool {
if len(r.msgs) > 0 || len(r.raftLog.unstableEntries()) > 0 || r.raftLog.hasNextEnts() {
return true
}
if len(r.readStates) != 0 {
return true
}
return false
}
@@ -231,3 +254,11 @@ func (rn *RawNode) ReportSnapshot(id uint64, status SnapshotStatus) {
func (rn *RawNode) TransferLeader(transferee uint64) {
_ = rn.raft.Step(pb.Message{Type: pb.MsgTransferLeader, From: transferee})
}
// ReadIndex requests a read state. The read state will be set in ready.
// Read State has a read index. Once the application advances further than the read
// index, any linearizable read requests issued before the read request can be
// processed safely. The read state will have the same rctx attached.
func (rn *RawNode) ReadIndex(rctx []byte) {
_ = rn.raft.Step(pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}})
}

118
vendor/github.com/coreos/etcd/raft/read_only.go generated vendored Normal file
View File

@@ -0,0 +1,118 @@
// Copyright 2016 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package raft
import pb "github.com/coreos/etcd/raft/raftpb"
// ReadState provides state for read only query.
// It's caller's responsibility to call ReadIndex first before getting
// this state from ready, It's also caller's duty to differentiate if this
// state is what it requests through RequestCtx, eg. given a unique id as
// RequestCtx
type ReadState struct {
Index uint64
RequestCtx []byte
}
type readIndexStatus struct {
req pb.Message
index uint64
acks map[uint64]struct{}
}
type readOnly struct {
option ReadOnlyOption
pendingReadIndex map[string]*readIndexStatus
readIndexQueue []string
}
func newReadOnly(option ReadOnlyOption) *readOnly {
return &readOnly{
option: option,
pendingReadIndex: make(map[string]*readIndexStatus),
}
}
// addRequest adds a read only reuqest into readonly struct.
// `index` is the commit index of the raft state machine when it received
// the read only request.
// `m` is the original read only request message from the local or remote node.
func (ro *readOnly) addRequest(index uint64, m pb.Message) {
ctx := string(m.Entries[0].Data)
if _, ok := ro.pendingReadIndex[ctx]; ok {
return
}
ro.pendingReadIndex[ctx] = &readIndexStatus{index: index, req: m, acks: make(map[uint64]struct{})}
ro.readIndexQueue = append(ro.readIndexQueue, ctx)
}
// recvAck notifies the readonly struct that the raft state machine received
// an acknowledgment of the heartbeat that attached with the read only request
// context.
func (ro *readOnly) recvAck(m pb.Message) int {
rs, ok := ro.pendingReadIndex[string(m.Context)]
if !ok {
return 0
}
rs.acks[m.From] = struct{}{}
// add one to include an ack from local node
return len(rs.acks) + 1
}
// advance advances the read only request queue kept by the readonly struct.
// It dequeues the requests until it finds the read only request that has
// the same context as the given `m`.
func (ro *readOnly) advance(m pb.Message) []*readIndexStatus {
var (
i int
found bool
)
ctx := string(m.Context)
rss := []*readIndexStatus{}
for _, okctx := range ro.readIndexQueue {
i++
rs, ok := ro.pendingReadIndex[okctx]
if !ok {
panic("cannot find corresponding read state from pending map")
}
rss = append(rss, rs)
if okctx == ctx {
found = true
break
}
}
if found {
ro.readIndexQueue = ro.readIndexQueue[i:]
for _, rs := range rss {
delete(ro.pendingReadIndex, string(rs.req.Entries[0].Data))
}
return rss
}
return nil
}
// lastPendingRequestCtx returns the context of the last pending read only
// request in readonly struct.
func (ro *readOnly) lastPendingRequestCtx() string {
if len(ro.readIndexQueue) == 0 {
return ""
}
return ro.readIndexQueue[len(ro.readIndexQueue)-1]
}

View File

@@ -98,6 +98,8 @@ func (ms *MemoryStorage) InitialState() (pb.HardState, pb.ConfState, error) {
// SetHardState saves the current HardState.
func (ms *MemoryStorage) SetHardState(st pb.HardState) error {
ms.Lock()
defer ms.Unlock()
ms.hardState = st
return nil
}
@@ -130,6 +132,9 @@ func (ms *MemoryStorage) Term(i uint64) (uint64, error) {
if i < offset {
return 0, ErrCompacted
}
if int(i-offset) >= len(ms.ents) {
return 0, ErrUnavailable
}
return ms.ents[i-offset].Term, nil
}
@@ -168,7 +173,13 @@ func (ms *MemoryStorage) ApplySnapshot(snap pb.Snapshot) error {
ms.Lock()
defer ms.Unlock()
// TODO: return ErrSnapOutOfDate?
//handle check for old snapshot being applied
msIndex := ms.snapshot.Metadata.Index
snapIndex := snap.Metadata.Index
if msIndex >= snapIndex {
return ErrSnapOutOfDate
}
ms.snapshot = snap
ms.ents = []pb.Entry{{Term: snap.Metadata.Term, Index: snap.Metadata.Index}}
return nil

View File

@@ -48,11 +48,23 @@ func max(a, b uint64) uint64 {
func IsLocalMsg(msgt pb.MessageType) bool {
return msgt == pb.MsgHup || msgt == pb.MsgBeat || msgt == pb.MsgUnreachable ||
msgt == pb.MsgSnapStatus || msgt == pb.MsgCheckQuorum || msgt == pb.MsgTransferLeader
msgt == pb.MsgSnapStatus || msgt == pb.MsgCheckQuorum
}
func IsResponseMsg(msgt pb.MessageType) bool {
return msgt == pb.MsgAppResp || msgt == pb.MsgVoteResp || msgt == pb.MsgHeartbeatResp || msgt == pb.MsgUnreachable
return msgt == pb.MsgAppResp || msgt == pb.MsgVoteResp || msgt == pb.MsgHeartbeatResp || msgt == pb.MsgUnreachable || msgt == pb.MsgPreVoteResp
}
// voteResponseType maps vote and prevote message types to their corresponding responses.
func voteRespMsgType(msgt pb.MessageType) pb.MessageType {
switch msgt {
case pb.MsgVote:
return pb.MsgVoteResp
case pb.MsgPreVote:
return pb.MsgPreVoteResp
default:
panic(fmt.Sprintf("not a vote message: %s", msgt))
}
}
// EntryFormatter can be implemented by the application to provide human-readable formatting