removeNode reduces the required quorum size, so some pending entries may
be able to commit after it is applied.
Discovered in cockroachdb/cockroach#3642
We need to be able to force an election (on one node) after creating a
new group (cockroachdb/cockroach#1384), but it is difficult to ensure
that our call to Campaign does not race with an election that may be
started by raft itself. A redundant call to Campaign should be a no-op
instead of a panic. (But the panic in becomeCandidate remains, because
we don't want to update the term or change the committed index in this
case)
sendApp accesses the storage several times. Perviously, we
assume that the storage will not be modified during the read
opeartions. The assumption is not true since the storage can
be compacted between the read operations. If a compaction
causes a read entries error, we should not painc. Instead, we
can simply retry the sendApp logic until succeed.
Each progress has a inflighs sliding window. When the progress
is in replicate state, inflights will control the sending speed
of the leader.
The leader can have at most maxInflight number of inflight
messages for each replicate progress. Receving a appResp moves
forward the sliding window. Heartbeat response free one
slot if the window is full.
limit the max size of entries sent per message.
Lower the cost at probing state as we limit the size per message;
lower the penalty when aggressively decrease to a too low next.
This addresses a problem that comes up in the cockroach tests,
in which the order of messages may lead to deadlocks (due to
the fact that we don't have regular heartbeat timers in most
of our tests).
Now that heartbeats are distinct from MsgApp{,Resp}, the retries
currently performed in stepLeader's MsgAppResp section are only
performed on an actual MsgAppResp (or a new MsgProp). This means
that it may take a long time to recover from a dropped MsgAppResp
in a quiet cluster.
This commit adds a dedicated heartbeat response message. This message
does not convey the follower's current log position because the
MsgHeartbeat does not include the leaders term and index. Upon receipt
of a heartbeat response, the leader may retry the latest MsgApp if it
believes the follower to be behind.
It is reasonable for the leader to wait for the reply before sending out the next
msgApp or msgSnap for the follower in bad path. Or the leader will send out useless
messages if the previous message is rejected or the previous message is a snapshot.
Especially for the snapshot case, the leader will be 100% to send out duplicate message
including the snapshot, which is a huge waste.
This commit implement a timeout based wait mechanism. The timeout for normal msgApp is a
heartbeatTimeout and the timeout for snapshot is electionTimeout(snapshot is larger). We
can implement a piggyback mechanism(application notifies the msg lost) in the future
if necessary.
stableTo should only mark the index stable if the term is matched. After raft sends out unstable
entries to application, raft makes progress without waiting for reply. When the appliaction
calls the stableTo to notify the entries up to "index" are stable, raft might have truncated
some entries before "index" due to leader lost. raft must verify the (index,term) of stableTo,
before marking the entries as stable.
* coreos/master:
rafthttp: fix import
raft: should not decrease match and next when handling out of order msgAppResp
Fix migration to allow snapshots to have the right IDs
add snapshotted integration test
fix test import loop
fix import loop, add set to types, and fix comments
etcdserver: autodetect v0.4 WALs and upgrade them to v0.5 automatically
wal: add a bench for write entry
rafthttp: add streaming server and client
dep: use vendored imports in codegangsta/cli
dep: bump golang.org/x/net/context
Conflicts:
etcdserver/server.go
etcdserver/server_test.go
migrate/snapshot.go
* coreos/master:
scripts: build-docker tag and use ENTRYPOINT
scripts: build-release add etcd-migrate
create .godir
raft: optimistically increase the next if the follower is already matched
raft: add handleHeartbeat handleHeartbeat commits to the commit index in the message. It never decreases the commit index of the raft state machine.
rafthttp: send takes raft message instead of bytes
*: add rafthttp pkg into test list
raft: include commitIndex in heartbeat
rafthttp: move server stats in raftHandler to etcdserver
*: etcdhttp.raftHandler -> rafthttp.RaftHandler
etcdserver: rename sender.go -> sendhub.go
*: etcdserver.sender -> rafthttp.Sender
Conflicts:
raft/log.go
raft/raft_paper_test.go
Compaction is now treated as an implementation detail of Storage
implementations; Node.Compact() and related functionality have been
removed. Ready.Snapshot is now used only for incoming snapshots.
A return value has been added to ApplyConfChange to allow applications
to track the node information that must be stored in the snapshot.
raftpb.Snapshot has been split into Snapshot and SnapshotMetadata, to
allow the full snapshot data to be read from disk only when needed.
raft.Storage has new methods Snapshot, ApplySnapshot, HardState, and
SetHardState. The Snapshot and HardState parameters have been removed
from RestartNode() and will now be loaded from Storage instead.
The only remaining difference between StartNode and RestartNode is that
the former bootstraps an initial list of Peers.
This is useful since we want to pipeline the appendEntry requests. Without
enabling optimistic increasing, the second pipelining appendEntry request
will include the entries the first one has already sent out. We decrease
the next directly to match if the leader receives a rejection for a matched
follower. This happens if one pipelining request get lost and following ones
arrives at the follower.