Commit Graph

92 Commits

Author SHA1 Message Date
Abhi
9b09439065 raft: Add descriptions for requested abort errors
Fixes: scylladb/scylladb#18902

Closes scylladb/scylladb#20291
2024-09-10 17:56:29 +02:00
Sergey Zolotukhin
c3e52ab942 raft: Invoke store_snapshot_descriptor with actually preserved items.
- raft_sys_table_storage::store_snapshot_descriptor now receives a number of
preserved items in the log, rather than _config.snapshot_trailing value;
- Incorrect check for truncated number of items in store_snapshot_descriptor
was removed.

Fixes scylladb/scylladb#16817
Fixes scylladb/scylladb#20080
2024-08-20 15:22:49 +02:00
Avi Kivity
aa1270a00c treewide: change assert() to SCYLLA_ASSERT()
assert() is traditionally disabled in release builds, but not in
scylladb. This hasn't caused problems so far, but the latest abseil
release includes a commit [1] that causes a 1000 insn/op regression when
NDEBUG is not defined.

Clearly, we must move towards a build system where NDEBUG is defined in
release builds. But we can't just define it blindly without vetting
all the assert() calls, as some were written with the expectation that
they are enabled in release mode.

To solve the conundrum, change all assert() calls to a new SCYLLA_ASSERT()
macro in utils/assert.hh. This macro is always defined and is not conditional
on NDEBUG, so we can later (after vetting Seastar) enable NDEBUG in release
mode.

[1] 66ef711d68

Closes scylladb/scylladb#20006
2024-08-05 08:23:35 +03:00
Gleb Natapov
9ebdb23002 raft: add more raft metrics to make debug easier 2024-07-01 10:55:22 +02:00
Kamil Braun
a441d06d6c raft: fsm: add details to on_internal_error_noexcept message
If we receive a message in the same term but from a different leader
than we expect, we print:
```
Got append request/install snapshot/read_quorum from an unexpected leader
```
For some reason the message did not include the details (who the leader
was and who the sender was) which requires almost zero effort and might
be useful for debugging. So let's include them.

Ref: scylladb/scylla-enterprise#4276

Closes scylladb/scylladb#19238
2024-06-12 17:29:42 +03:00
Kefu Chai
57ede58a64 raft: add fmt::formatter for raft::fsm
before this change, we rely on the default-generated fmt::formatter
created from operator<<, but fmt v10 dropped the default-generated
formatter.

in this change, we define formatters for `raft::fsm`, and drop its
operator<<.

Refs #13245

Signed-off-by: Kefu Chai <kefu.chai@scylladb.com>

Closes scylladb/scylladb#17414
2024-02-20 09:02:02 +02:00
Kamil Braun
1824c12975 raft: remove empty() from fsm_output
Nobody remembered to keep this function up to date when adding stuff to
`fsm_output`.

Turns out that it's not being used by any Raft logic but only in some
tests. That use case can now be replaced with `fsm::has_output()` which
is also being used by `raft::server` code.
2024-01-23 16:48:28 +01:00
Kamil Braun
3e6b4910a6 raft: move poll_output() from fsm to server
`server` was the only user of this function and it can now be
implemented using `fsm`'s public interface.

In later commits we'll extend the logic of `io_fiber` to also subscribe
to other events, triggered by `server` API calls, not only to outputs
from `fsm`.
2024-01-18 18:07:52 +01:00
Kamil Braun
95b6a60428 raft: move _sm_events from fsm to server
In later commits we will use it to wake up `io_fiber` directly from
`raft::server` based on events generated by `raft::server` itself -- not
only from events generated by `raft::fsm`.

`raft::fsm` still obtains a reference to the condition variable so it
can keep signaling it.
2024-01-18 18:07:44 +01:00
Kamil Braun
a83e04279e raft: fsm: remove constructor used only in tests
This constructor does not provide persisted commit index. It was only
used in tests, so move it there, to the helper `fsm_debug` which
inherits from `fsm`.

Test cases which used `fsm` directly instead of `fsm_debug` were
modified to use `fsm_debug` so they can access the constructor.
`fsm_debug` doesn't change the behavior of `fsm`, only adds some helper
members. This will be useful in following commits too.
2024-01-18 18:07:17 +01:00
Kamil Braun
f6d43779af raft: fsm: extract has_output()
Also use the more efficient coroutine-specific
`condition_variable::when` instead of `wait`.
2024-01-18 18:06:27 +01:00
Kamil Braun
dccfd09d83 raft: pass max_trailing_entries through fsm_output to store_snapshot_descriptor
This parameter says how many entries at most should be left trailing
before the snapshot index. There are multiple places where this
decision is made:
- in `applier_fiber` when the server locally decides to take a snapshot
  due to log size pressure; this applies to the in-memory log
- in `fsm::step` when the server received an `install_snapshot` message
  from the leader; this also applies to the in-memory log
- and in `io_fiber` when calling `store_snapshot_descriptor`; this
  applies to the on-disk log.

The logic of how many entries should be left trailing is calculated
twice:
- first, in `applier_fiber` or in `fsm::step` when truncating the
  in-memory log
- and then again as the snapshot descriptor is being persisted.

The logic is to take `_config.snapshot_trailing` for locally generated
snapshots (coming from `applier_fiber`) and `0` for remote snapshots
(from `fsm::step`).

But there is already an error injection that changes the behavior of
`applier_fiber` to leave `0` trailing entries. However, this doesn't
affect the following `store_snapshot_descriptor` call which still uses
`_config.snapshot_trailing`. So if the server got restarted, the entries
which were truncated in-memory would get "revived" from disk.
Fortunately, this is test-only code.

However in future commits we'd like to change the logic of
`applier_fiber` even further. So instead of having a separate
calculation of trailing entries inside `io_fiber`, it's better for it to
use the number that was already calculated once. This number is passed to
`fsm::apply_snapshot` (by `applier_fiber` or `fsm::step`) and can then
be received by `io_fiber` from `fsm_output` to use it inside
`store_snapshot_descriptor`.
2024-01-18 18:05:45 +01:00
Harsh Soni
84ea2f5066 raft: fsm: add empty check for max_read_id_with_quorum
Updated the empty() function in the struct fsm_output to include the
max_read_id_with_quorum field when checking whether the fsm output is
empty or not. The change was made in order maintain consistency with the
codebase and adding completeness to the empty check. This change has no
impact on other parts of the codebase.

Closes #13656
2023-04-27 16:04:58 +02:00
Gleb Natapov
2fc8e13dd8 raft: add server::wait_for_state_change() function
Add a function that allows waiting for a state change of a raft server.
It is useful for a user that wants to know when a node becomes/stops
being a leader.

Message-Id: <20230316112801.1004602-4-gleb@scylladb.com>
2023-03-20 11:31:55 +01:00
Kefu Chai
b926105eae raft: reference this explicitly
Clang complains that the captured `this` is not used, like
```
/home/kefu/dev/scylladb/raft/fsm.hh:644:21: error: lambda capture 'this' is not used [-Werror,-Wunused-lambda-capture]
    auto visitor = [this, from, msg = std::move(msg)](const auto& state) mutable {
                    ^
/home/kefu/dev/scylladb/raft/server.cc:738:11: note: in instantiation of function template specialization 'raft::fsm::step<raft::append_request>' requested here
    _fsm->step(from, std::move(append_request));
          ^
```
but `step(..)` is a non-static member function of `fsm`, so `this`
is actually used. to silence Clang's warning, let's just reference it
explicitly.

Signed-off-by: Kefu Chai <kefu.chai@scylladb.com>
2023-02-28 21:56:55 +08:00
Petr Gusev
b34dfed307 raft server, release memory if add_entry throws
We consume memory from semaphore in add_entry_on_leader, but never release it if add_entry throws.
2022-09-27 12:02:34 +04:00
Petr Gusev
27e60ecbf4 raft server, log size limit in bytes
Before this patch we could get an OOM if we
received several big commands. The number of
commands was small, but their total size
in bytes was large.

snapshot_trailing_size is needed to guarantee
progress. Without this limit the fsm could
get stuck if the size of the next item is
greater than max_log_size - (size of trailing entries).
2022-09-26 13:10:10 +04:00
Petr Gusev
210d9dd026 raft: fix snapshots leak
applier_fiber could create multiple snapshots between
io_fiber run. The fsm_output.snp variable was
overwritten by applier_fiber and io_fiber didn't drop
the previous snapshot.

In this patch we introduce the variable
fsm_output.snps_to_drop, store in it
the current snapshot id before applying
a new one, and then sequentially drop them in
io_fiber after storing the last snapshot_descriptor.

_sm_events.signal() is added to fsm::apply_snapshot,
since this method mutates the _output and thus gives a
reason to run io_fiber.

The new test test_frequent_snapshotting demonstrates
the problem by causing frequent snapshots and
setting the applier queue size to one.

Closes #11530
2022-09-21 12:46:26 +02:00
Kamil Braun
efad6fe9b4 raft: server: handle aborts when waiting for config entry to commit
Changing configuration involves two entries in the log: a 'joint
configuration entry' and a 'non-joint configuration entry'. We use
`wait_for_entry` to wait on the joint one. To wait on the non-joint one,
we use a separate promise field in `server`. This promise wasn't
connected to the `abort_source` passed into `set_configuration`.

The call could get stuck if the server got removed from the
configuration and lost leadership after committing the joint entry but
before committing the non-joint one, waiting on the promise. Aborting
wouldn't help. Fix this by subscribing to the `abort_source` in
resolving the promise exceptionally.

Furthermore, make sure that two `set_configuration` calls don't step on
each other's toes by one setting the other's promise. To do that, reset
the promise field at the end of `set_configuration` and check that it's
not engaged at the beginning.

Fixes #11288.
2022-08-23 13:14:29 +02:00
Kamil Braun
daf9c53bb8 raft: split can_vote field from server_address to separate struct
Whether a server can vote in a Raft configuration is not part of the
address. `server_address` was used in many context where `can_vote` is
irrelevant.

Split the struct: `server_address` now contains only `id` and
`server_info` as it did before `can_vote` was introduced. Instead we
have a `config_member` struct that contains a `server_address` and the
`can_vote` field.

Also remove an "unsafe" constructor from `server_address` where `id` was
provided but `server_info` was not. The constructor was used for tests
where `server_info` is irrelevant, but it's important not to forget
about the info in production code. The constructor was used for two
purposes:
- Invoking set operations such as `contains`. To solve this we use C++20
  transparent hash and comparator functions, which allow invoking
  `contains` and similar functions by providing a different key type (in
  this case `raft::server_id` in set of addresses, for example).
- constructing addresses without `info`s in tests. For this we provide
  helper functions in the test helpers module and use them.
2022-07-18 18:22:10 +02:00
Gleb Natapov
7f26a8eef5 raft: actively search for a leader if it is not known for a tick duration
For a follower to forward requests to a leader the leader must be known.
But there may be a situation where a follower does not learn about
a leader for a while. This may happen when a node becomes a follower while its
log is up-to-date and there are no new entries submitted to raft. In such
case the leader will send nothing to the follower and the only way to
learn about the current leader is to get a message from it. Until a new
entry is added to the raft's log a follower that does not know who the
leader is will not be able to add entries. Kind of a deadlock. Note that
the problem is specific to our implementation where failure detection is
done by an outside module. In vanilla raft a leader sends messages to
all followers periodically, so essentially it is never idle.

The patch solves this by broadcasting specially crafted append reject to all
nodes in the cluster on a tick in case a leader is not known. The leader
responds to this message with an empty append request which will cause the
node to learn about the leader. For optimisation purposes the patch
sends the broadcast only in case there is actually an operation that
waits for leader to be known.

Fixes #10379
2022-04-25 14:51:22 +02:00
Kamil Braun
0f0d75fd66 raft: server: translate semaphore_aborted to request_aborted 2022-03-29 15:10:29 +02:00
Gleb Natapov
a1604aa388 raft: make raft requests abortable
This patch adds an ability to pass abort_source to raft request APIs (
add_entry, modify_config) to make them abortable. A request issuer not
always want to wait for a request to complete. For instance because a
client disconnected or because it no longer interested in waiting
because of a timeout. After this patch it can now abort waiting for such
requests through an abort source. Note that aborting a request only
aborts the wait for it to complete, it does not mean that the request
will not be eventually executed.

Message-Id: <YjHivLfIB9Xj5F4g@scylladb.com>
2022-03-16 18:38:01 +01:00
Gleb Natapov
579dcf187a raft: allow an option to persist commit index
Raft does not need to persist the commit index since a restarted node will
either learn it from an append message from a leader or (if entire cluster
is restarted and hence there is no leader) new leader will figure it out
after contacting a quorum. But some users may want to be able to bring
their local state machine to a state as up-to-date as it was before restart
as soon as possible without any external communication.

For them this patch introduces new persistence API that allows saving
and restoring last seen committed index.

Message-Id: <YfFD53oS2j1My0p/@scylladb.com>
2022-01-26 14:06:39 +01:00
Avi Kivity
fcb8d040e8 treewide: use Software Package Data Exchange (SPDX) license identifiers
Instead of lengthy blurbs, switch to single-line, machine-readable
standardized (https://spdx.dev) license identifiers. The Linux kernel
switched long ago, so there is strong precedent.

Three cases are handled: AGPL-only, Apache-only, and dual licensed.
For the latter case, I chose (AGPL-3.0-or-later and Apache-2.0),
reasoning that our changes are extensive enough to apply our license.

The changes we applied mechanically with a script, except to
licenses/README.md.

Closes #9937
2022-01-18 12:15:18 +01:00
Konstantin Osipov
e3751068fe raft: (server) allow adding entries/modify config on a follower
Implement an RPC to forward add_entry calls from the follower
to leader. Bounce & retry in case of not_a_leader.
Do not retry in case of uncertainty - this can lead to adding
duplicate entries.

The feature is added to core Raft since it's needed by
all current clients - both topology and schema changes.

When forwarding an entry to a remote leader we may get back
a term/index pair that conflicts (has the same index, but is with
a higher term) with a local entry we're still waiting on.

This can happen, e.g. because there was a leader change and the
log was truncated, but we still haven't got the append_entries
RPC from the new leader, still haven't truncated the log locally,
still haven't aborted all the local waits for truncated entries.

Only remove the offending entry from the wait list and abort it.
There may be entries labeled with an older term to the right (with
higher commit index) of the conflicting entry. However, finding them,
would require a linear scan. If we allow it, we may end up doing this
linear scan for *every* conflicting entry during the transition
period, which brings us to N^2 complexity of this step. At the
same time, as soon as append_entries that commits a higher-term
entry with the same index reaches the follower, the waits
for the respective truncated entry will be aborted anyway (see
notify_waiters() which sets dropped_entry exception), so the scan
is unnecessary.

Similarly to being able to add entries, allow to modify
Raft group configuration on a follower. The implementation
works the same way as adding entries - forwards the command
to the leader.

Now that add_entry() or modify_config never throws not_a_leader,
it's more likely to throw timed_out_error, e.g. in case the
network is partitioned. Previously it was only possible due to a
semaphore wait timeout, and this scenario was not tested.
Handle timed_out_error on RPC level to let the existing tests
(specifically the randomized nemesis test) pass.
2021-11-25 11:50:38 +03:00
Gleb Natapov
7aac6c2086 raft: rename rpc_configuration to configuration in fsm output
The filed is generic and used not only for rpc configuration now.
2021-11-09 15:16:57 +02:00
Kamil Braun
bf823e34a4 raft: disable sticky leadership rule
The Raft PhD presents the following scenario.
When we remove a server from the cluster configuration, it does not
receive the configuration entry which removes it (because the leader
appending this entry uses that entry's configuration to decide to which
servers to send the entry to, and the entry does not contain the removed
server). Therefore the server keeps believing it is a member but does
not receive heartbeats from leaders in the new configuration. Therefore
it will keep becoming a candidate, causing existing leaders to step
down, harming availability. With many such candidates the cluster may
even stop being able to proceed at all. We call such servers
"disruptive".

More concretely, consider the following example, adapted from the PhD for
joint configuration changes (the original PhD considered a different
algorithm which can only add/remove one server at once):
Let C_old = {A, B, C, D}, C_new = {B, C, D}, and C_joint be the joint
configuration (C_old, C_new). D is the leader. D managed to append
C_joint to every server and commit it. D appends C_new. At this point, D
stops sending heartbeats to A because C_new does not contain A, but A's
last entry is still C_joint, so it still has the ability to become a
candidate. A can now become a candidate and cause D, or any other leader
in C_new, to step down. Even if D manages to commit C_new, A can keep
disrupting the cluster until it is shut down.

Prevoting changes the situation, which the authors admit. The "even if"
above no longer applies: if D manages to commit C_new, or just append it
to a majority of C_new, then A won't be able to succeed in the prevote
phase because a majority of servers in C_new has a longer log than A
(and A must obtain a prevote from a majority of servers in C_new because
A is in C_joint which contains C_new). But the authors continue to argue
that disruptions can still occur during the small period where C_new is
only appended on D but not yet on a majority of C_new. As they say:
"we also did not want to assume that a leader will reliably replicate
entries fast enough to move past the scenario (...) quickly; that might
have worked in practice, but it depends on stronger assumptions that we
prefer to avoid about the performance (...) of replicating log entries".
One could probably try debunking this by saying that if entries take
longer to replicate than the election timeout we're in much bigger
trouble, but nevermind.

In any case, the authors propose a solution which we call "sticky
leadership". A server will not grant a vote to a candidate if it has
recently received a heartbeat from the currently known leader, even if
the candidate's term is higher. In the above example, servers in C_new
would not grant votes to A as long as D keeps sending them heartbeats,
thus A is no longer disruptive.

In our case the situation is a bit
different: in original Raft, "heartbeats" have a very specific meaning
- they are append_entries requests (possibly empty) sent by leaders.
Thus if a node stops being a leader it stops sending heartbeats;
similarly, if a node leaves the configuration, it stops receiving
heartbeats from others still in the configuration. We instead use a
"shared failure detector" interface, where nodes may still consider
other nodes alive regardless of their configuration/leadership
situation, as part of the general "MultiRaft" framework.

This pretty much invalidates the original argument, as seen on
the above example: A will still consider D alive, thus it won't become
a candidate.

Shared failure detector combined with sticky leadership actually makes
the situation worse - it may cause cluster unavailability in certain
scenarios (fortunately not a permanent one, it can be solved with server
restarts, for example). Randomized nemesis testing with reconfigurations
found the following scenario:
Let C1 = {A, B, C}, C2 = {A}, C3 = {B, C}. We start from configuration
C1, B is the leader. B commits joint (C1, C2), then new C2
configuration. Note that C does not learn about the last entry
(since it's not part of C2) but it keeps believing that B is alive,
so it keeps believing that B is the leader.
We then partition {A} from {B, C}. A appends (C2, C3) joint
configuration to its log. It's not able to append it to B or C due to
the partition. The partition holds long enough for A to revert to
candidate state (or we may restart A at this point). Eventually the
partition resolves. The only node which can become a candidate now is A:
C does not become a candidate because it keeps believeing that B is the
leader, and B does not become a candidate because it saw the C2
non-joint entry being committed. However, A won't become a leader
because C won't grant it a vote due to the sticky leadership rule.
The cluster will remain unavailable until e.g. C is restarted.

Note that this scenario requires allowing configuration changes which
remove and then readd the same servers to the configuration. One may
wonder if such reconfigurations should be allowed, but there doesn't
seem to be any example of them breaking safety of Raft (and the PhD
doesn't seem to mention them at all; perhaps it implicitly accepts
them). It is unknown whether a similar scenario may be produced without
such reconfigurations.

In any case, disabling sticky leadership resolves the problem, and it is
the last currently known availability problem found in randomized
nemesis testing. There is no reason to keep this extension, both because
the original Raft authors' argument does not apply for shared failure
detector, and because one may even argue with the authors in vanilla
Raft given that prevoting is enabled (see end of third paragraph of this
commit message).
Message-Id: <20210921153741.65084-1-kbraun@scylladb.com>
2021-09-26 11:09:01 +03:00
Gleb Natapov
ce40b01b07 raft: rename snapshot into snapshot_descriptor
The snapshot structure does not contain the snapshot itself but only
refers to it trough its id. Rename it to snapshot_descriptor for clarity.
2021-08-29 12:53:03 +03:00
Gleb Natapov
f9f859ac40 raft: fix local snapshot detection
The code assumes that the snapshot that was taken locally is never
applied. Currently logic to detect that is flawed. It relies on an
id of a most recently applied snapshot (where a locally taken snapshot
is considered to be applied as well). But if between snapshot creation
and the check another local snapshot is taken ids will not match.

The patch fixes this by propagating locality information together with
the snapshot itself.
2021-08-29 12:53:03 +03:00
Gleb Natapov
03a266d73b raft: make read_barrier work on a follower as well as on a leader
This patch implements RAFT extension that allows to perform linearisable
reads by accessing local state machine. The extension is described
in section 6.4 of the PhD. To sum it up to perform a read barrier on
a follower it needs to asks a leader the last committed index that it
knows about. The leader must make sure that it is still a leader before
answering by communicating with a quorum. When follower gets the index
back it waits for it to be applied and by that completes read_barrier
invocation.

The patch adds three new RPC: read_barrier, read_barrier_reply and
execute_read_barrier_on_leader. The last one is the one a follower uses
to ask a leader about safe index it can read. First two are used by a
leader to communicate with a quorum.
2021-08-25 08:57:13 +03:00
Gleb Natapov
376785042f raft: make fsm::current_leader() public
Later patch will call it from server class.
2021-08-25 08:19:25 +03:00
Kamil Braun
907672622f raft: fsm: update _commit_idx when applying snapshot
All entries up to snapshot.idx must obviously be committed, so why not
update _commit_idx to reflect that.

With this we get a useful invariant:
`_log.get_snapshot().idx <= _commit_idx`.
For example, when checking whether the latest active configuration is
committed, it should be enough to compare the configuration index to the
commit index. Without the invariant we would need a special case if the
latest configuration comes from a snapshot.
2021-08-06 12:43:07 +02:00
Kamil Braun
c6563220b0 raft: store cluster configuration when taking snapshots
We add a function `log_last_conf_before(index_t)` to `fsm` which, given
an index greater than the last snapshot index, returns the configuration
at this index, i.e. the configuration of the last configuration entry
before this index.

This function is then used in `applier_fiber` to obtain the correct
configuration to be stored in a snapshot.

In order to ensure that the configuration can be obtained, i.e. the
index we're looking at is not smaller than the last snapshot index, we
strengthen the conditions required for taking a snapshot: we check that
`_fsm` has not yet applied a snapshot at a larger index (which it may
have due to a remote snapshot install request). This also causes fewer
unnecessary snapshots to be taken in general.
2021-08-06 12:00:32 +02:00
Kamil Braun
f050d3682c raft: fsm: stronger check for outdated remote snapshots
We must not apply remote snapshots with commit indexes smaller than our
local commit index; this could result in out-of-order command
application to the local state machine replica, leading to
serializability violations.
Message-Id: <20210805112736.35059-1-kbraun@scylladb.com>
2021-08-05 14:29:50 +02:00
Gleb Natapov
7261c2c93e raft: return a correct leader when leaving leader state
When a leader moves to a follower state it aborts all requests that are
waiting on an admission semaphore with not_a_leader exception. But
currently it specifies itself as a new leader since abortion happens
before the fsm state changes to a follower. The patch fixes this by
destroying leader state after fsm state already changed to be a
follower.

Message-Id: <YPbI++0z5ZPV9pKb@scylladb.com>
2021-07-21 00:42:39 +02:00
Avi Kivity
332b5c395f raft: avoid changing meaning of a symbol inside a class
The construct

struct q {
    a a;
};

Changes the meaning of `a` from a type to a data member. gcc dislikes
it and I agree. Fully qualify the type name to avoid an error.
2021-07-11 18:16:21 +03:00
Gleb Natapov
ed49d29473 raft: allow to initiate leader stepdown process
Sometimes an ability to force a leader change is needed. For instance
if a node that is currently serving as a leader needs to be brought
down for maintenance. If it will be shutdown without leadership
transfer the cluster will be unavailable for leader election timeout at
least.

We already have a mechanism to transfer the leadership in case an active
leader is removed. The patch exposes it as an external interface with a
timeout period. If a node is still a leader after the timeout the
operation will fail.
2021-06-22 14:36:42 +03:00
Alejo Sanchez
5c8092cf42 raft: fix election with disruptive candidate
This patch also fixes rare hangs in debug mode for drops_04 without
prevote.

Branch URL: https://github.com/alecco/scylla/tree/raft-fixes-05-v2-dueling

Tests: unit ({dev}), unit ({debug}), unit ({release})

Changes in v2:
    - Fixed commit message                               @kostja

Whithout prevote, a node disconnected for long enough becomes candidate.
While disconnected (A) it keeps increasing its term.
When it rejoins it disrupts the current leader (C) which steps down due
to the higher term in (A)'s append_entries_reply and (C) also increases
its term.

Meanwhile followers (B) and (D) don't know (C) stepped down but see it
alive according to the current failure detecture implementation, and
also (A) has shorter log than them.
So they reject (A)'s vote requests (Raft 4.2.3 Disruptive servers).

Then (C) rejects voting for (A) because it has shorter log.
And (C) becomes candidate but even though (A) votes for (C), the
previous followers (B) and (D) ignore a vote request while leader (C) is
still alive and election timeout has not passed.

(A) and (C) alone can't reach quorum 2/4. So elections never succeed.

This patch addresses this problem by making followers not ignore vote
requests from who they think is the current leader even though
election timout was not reached.

As @kostja noted, if failure detector would consider a leader alive only
as long as it sends heartbeats (append requests) this patch is no longer
needed.

Signed-off-by: Alejo Sanchez <alejo.sanchez@scylladb.com>
Message-Id: <20210611172734.254757-1-alejo.sanchez@scylladb.com>
2021-06-14 11:07:38 +02:00
Konstantin Osipov
eaf32f2c3c raft: (testing) test receiving a confchange in a snapshot 2021-06-11 17:16:56 +03:00
Avi Kivity
a55b434a2b treewide: extent copyright statements to present day 2021-06-06 19:18:49 +03:00
Gleb Natapov
f5a54d6c05 raft: move ELECTION_TIMEOUT definition to a public header
Move ELECTION_TIMEOUT definition to be visible to outside modules.
2021-06-06 19:18:49 +03:00
Tomasz Grabiec
0fdd2f8217 Merge "raft: fsm cleanups" from Gleb
* scylla-dev/raft-cleanup-v1:
  raft: drop _leader_progress tracking from the tracker
  raft: move current_leader into the follower state
  raft: add some precondition checks
2021-05-14 17:24:59 +02:00
Alejo Sanchez
68f69671b5 raft: style: test optionals directly
Avoid using has_value() and test optional directly

Signed-off-by: Alejo Sanchez <alejo.sanchez@scylladb.com>
Message-Id: <20210512142018.297203-2-alejo.sanchez@scylladb.com>
2021-05-12 20:39:52 +02:00
Gleb Natapov
78c5a72b32 raft: drop _leader_progress tracking from the tracker
The tracker maintains a separate pointer to current leader progress,
but all this complexity is not needed because the tracker already have
find() function that can either find a leader's progress by id or return
null. Removing the tracking simplifies code and make going out of sync
(which is always a possibility if a state is maintained in two different
places) impossible.
2021-05-09 13:55:55 +03:00
Gleb Natapov
1245736776 raft: move current_leader into the follower state
Only when fsm is in the follower state current_leader has any meaning.
In the leader state a node is always its own follower and in a candidate
state there is no leader. To make sure that the current_leader value
cannot be out of sync with fsm state move it into the follower state.
2021-05-09 13:55:55 +03:00
Gleb Natapov
0634674aef raft: add some precondition checks
Check that fsm does not process messages from itself and that it does
not tries to become its own follower.
2021-05-07 08:04:16 +03:00
Kamil Braun
4c95277619 raft: fsm: fix assertion failure on stray rejects
When probes are sent over a slow network, the leader would send
multiple probes to a lagging follower before it would get a
reject response to the first probe back. After getting a reject, the
leader will be able to correctly position `next_idx` for that
follower and switch to pipeline mode. Then, an out of order reject
to a now irrelevant probe could crash the leader, since it would
effectively request it to "rewind" its `match_idx` for that
follower, and the code asserts this never happens.

We fix the problem by strengthening `is_stray_reject`. The check that
was previously only made in `PIPELINE` case
(`rejected.non_matching_idx <= match_idx`) is now always performed and
we add a new check: `rejected.last_idx < match_idx`. We also strengthen
the assert.

The commit improves the documentation by explaining that
`is_stray_reject` may return false negatives.  We also precisely state
the preconditions and postconditions of `is_stray_reject`, give a more
precise definition of `progress.match_idx`, argue how the
postconditions of `is_stray_reject` follow from its preconditions
and Raft invariants, and argue why the (strengthened) assert
must always pass.
Message-Id: <20210423173117.32939-1-kbraun@scylladb.com>
2021-04-27 01:07:22 +02:00
Gleb Natapov
28add88a1f raft: do not assert when receiving unexpected messages in a leader state
Current code assert when it gets InstallSnapshot/AppendRequest in a leader
state and the term in the message is equal current term. It is true that
such messages cannot be received if the protocol works correctly, but we
should not crash on a network input nonetheless.
2021-04-04 11:33:35 +03:00
Gleb Natapov
995cd1c8a7 raft: use existing function to check if election timeout elapsed
is_past_election_timeout() repeats the calculation that
election_elapsed() is doing. Use existing function instead.
2021-04-04 11:33:35 +03:00