Also rename hasNextEnts to hasNextCommittedEnts.
Also rename maxNextEntsSize to maxNextCommittedEntsSize.
Pure refactor.
Signed-off-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
These are nonsensical and a network implementation is not required
to handle them correctly, so panic instead of sending them out.
Signed-off-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Leader only acks to itself on `(*raft).advance` so we have to
make this test a bit more like the real thing.
Signed-off-by: Tobias Grieger <tobias.b.grieger@gmail.com>
In the test `testNonleaderElectionTimeoutRandomized`, the possible values of election timeout randomized out should be 10-19(inclusive), but the current test only tests the possible values as 11-19(inclusive), which is incorrect here. We need to expand the scope of the tests to ensure robustness, and also to make the logic clearer.
Signed-off-by: Zike Yang <zike@apache.org>
This change makes the etcd package compatible with the existing Go
ecosystem for module versioning.
Used this tool to update package imports:
https://github.com/KSubedi/gomove
Mechanically extract `progressTracker`, `Progress`, and `inflights`
to their own package named `tracker`. Add lots of comments in the
progress, and take the opportunity to rename and clarify various
fields.
The previous code was using the proto-generated `Size()` method to
track the size of an incoming proposal at the leader. This includes
the Index and Term, which were mutated after the call to `Size()`
when appending to the log. Additionally, it was not taking into
account that an ignored configuration change would ignore the
original proposal and append an empty entry instead.
As a result, a fully committed Raft group could end up with a non-
zero tracked uncommitted Raft log counter that would eventually hit
the ceiling and drop all future proposals indiscriminately. It would
also immediately imply that proposals exceeding the threshold alone
would get refused (as the "first uncommitted proposal" gets special
treatment and is always allowed in).
Track only the size of the payload actually appended to the Raft log
instead.
For context, see:
https://github.com/cockroachdb/cockroach/issues/31618#issuecomment-431374938
`raft.Step` already ensures that when `m.Term > r.Term`,
candidate reverts back to follower with its term being
reset with `m.Term`, thus it's always true that
`m.Term == r.Term` in `stepCandidate`.
This just makes `r.becomeFollower` calls consistent.
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
Some tests were starting nodes with a non-empty log but a term of zero,
which cannot happen in the real world. This was affecting the final term
being tested in TestLeaderElection.
We recently changed the randomized election timeout from (et, 2*et-1] tp
[et, 2*et-2], where et is user set election timeout.
So 2*et might trigger two elections instead of one. We need to fix the test
code accordingly.
Thanks for Tikv guys for finding this issue. We probably need to randomize
etcd/raft test more.
Follower should not reject the append message with a smaller index than its commit
index. Or it will trigger the leader's resending logic, which might have a high cost.
In code outside the raft package, we cannot call raft.bcastHeartbeat
directly. Instead, to control heartbeats we set heartbeatInterval to 1
and call Tick().
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:
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
The first entry in the log is a dummy which is used for matchTerm
but may not have an actual payload. This change permits Storage
implementations to treat this term value specially instead of
storing it as a dummy Entry.
Storage.FirstIndex() no longer includes the term-only entry.
This reverses a recent decision to create entry zero as initially
unstable; Storage implementations are now required to make
Term(0) == 0 and the first unstable entry is now index 1.
stableTo(0) is no longer allowed.
This entry is now persisted through the normal flow instead of appearing
in the stored log at creation time. This is how things worked before
the Storage interface was introduced. (see coreos/etcd#1689)