]> git.proxmox.com Git - mirror_ovs.git/blame - ovsdb/raft.c
ovsdb_monitor: Fix style of prototypes.
[mirror_ovs.git] / ovsdb / raft.c
CommitLineData
1b1d2e6d
BP
1/*
2 * Copyright (c) 2017, 2018 Nicira, Inc.
3 *
4 * Licensed under the Apache License, Version 2.0 (the "License");
5 * you may not use this file except in compliance with the License.
6 * You may obtain a copy of the License at:
7 *
8 * http://www.apache.org/licenses/LICENSE-2.0
9 *
10 * Unless required by applicable law or agreed to in writing, software
11 * distributed under the License is distributed on an "AS IS" BASIS,
12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 * See the License for the specific language governing permissions and
14 * limitations under the License.
15 */
16
17#include <config.h>
18
19#include "raft.h"
20#include "raft-private.h"
21
22#include <errno.h>
23#include <unistd.h>
24
25#include "hash.h"
26#include "jsonrpc.h"
27#include "lockfile.h"
28#include "openvswitch/dynamic-string.h"
29#include "openvswitch/hmap.h"
30#include "openvswitch/json.h"
31#include "openvswitch/list.h"
32#include "openvswitch/poll-loop.h"
33#include "openvswitch/vlog.h"
34#include "ovsdb-error.h"
35#include "ovsdb-parser.h"
36#include "ovsdb/log.h"
37#include "raft-rpc.h"
38#include "random.h"
39#include "socket-util.h"
40#include "stream.h"
41#include "timeval.h"
42#include "unicode.h"
43#include "unixctl.h"
44#include "util.h"
45#include "uuid.h"
46
47VLOG_DEFINE_THIS_MODULE(raft);
48
49/* Roles for a Raft server:
50 *
51 * - Followers: Servers in touch with the current leader.
52 *
53 * - Candidate: Servers unaware of a current leader and seeking election to
54 * leader.
55 *
56 * - Leader: Handles all client requests. At most one at a time.
57 *
58 * In normal operation there is exactly one leader and all of the other servers
59 * are followers. */
60enum raft_role {
61 RAFT_FOLLOWER,
62 RAFT_CANDIDATE,
63 RAFT_LEADER
64};
65
66/* A connection between this Raft server and another one. */
67struct raft_conn {
68 struct ovs_list list_node; /* In struct raft's 'conns' list. */
69 struct jsonrpc_session *js; /* JSON-RPC connection. */
70 struct uuid sid; /* This server's unique ID. */
71 char *nickname; /* Short name for use in log messages. */
72 bool incoming; /* True if incoming, false if outgoing. */
73 unsigned int js_seqno; /* Seqno for noticing (re)connections. */
74};
75
76static void raft_conn_close(struct raft_conn *);
77
78/* A "command", that is, a request to append an entry to the log.
79 *
80 * The Raft specification only allows clients to issue commands to the leader.
81 * With this implementation, clients may issue a command on any server, which
82 * then relays the command to the leader if necessary.
83 *
84 * This structure is thus used in three cases:
85 *
86 * 1. We are the leader and the command was issued to us directly.
87 *
88 * 2. We are a follower and relayed the command to the leader.
89 *
90 * 3. We are the leader and a follower relayed the command to us.
91 */
92struct raft_command {
93 /* All cases. */
94 struct hmap_node hmap_node; /* In struct raft's 'commands' hmap. */
95 unsigned int n_refs; /* Reference count. */
96 enum raft_command_status status; /* Execution status. */
97
98 /* Case 1 only. */
99 uint64_t index; /* Index in log (0 if being relayed). */
100
101 /* Cases 2 and 3. */
102 struct uuid eid; /* Entry ID of result. */
103
104 /* Case 2 only. */
105 long long int timestamp; /* Issue or last ping time, for expiration. */
106
107 /* Case 3 only. */
108 struct uuid sid; /* The follower (otherwise UUID_ZERO). */
109};
110
111static void raft_command_complete(struct raft *, struct raft_command *,
112 enum raft_command_status);
113
114static void raft_complete_all_commands(struct raft *,
115 enum raft_command_status);
116
117/* Type of deferred action, see struct raft_waiter. */
118enum raft_waiter_type {
119 RAFT_W_ENTRY,
120 RAFT_W_TERM,
121 RAFT_W_RPC,
122};
123
124/* An action deferred until a log write commits to disk. */
125struct raft_waiter {
126 struct ovs_list list_node;
127 uint64_t commit_ticket;
128
129 enum raft_waiter_type type;
130 union {
131 /* RAFT_W_ENTRY.
132 *
133 * Waits for a RAFT_REC_ENTRY write to our local log to commit. Upon
134 * completion, updates 'log_synced' to indicate that the new log entry
135 * or entries are committed and, if we are leader, also updates our
136 * local 'match_index'. */
137 struct {
138 uint64_t index;
139 } entry;
140
141 /* RAFT_W_TERM.
142 *
143 * Waits for a RAFT_REC_TERM or RAFT_REC_VOTE record write to commit.
144 * Upon completion, updates 'synced_term' and 'synced_vote', which
145 * triggers sending RPCs deferred by the uncommitted 'term' and
146 * 'vote'. */
147 struct {
148 uint64_t term;
149 struct uuid vote;
150 } term;
151
152 /* RAFT_W_RPC.
153 *
154 * Sometimes, sending an RPC to a peer must be delayed until an entry,
155 * a term, or a vote mentioned in the RPC is synced to disk. This
156 * waiter keeps a copy of such an RPC until the previous waiters have
157 * committed. */
158 union raft_rpc *rpc;
159 };
160};
161
162static struct raft_waiter *raft_waiter_create(struct raft *,
163 enum raft_waiter_type,
164 bool start_commit);
165static void raft_waiters_destroy(struct raft *);
166
167/* The Raft state machine. */
168struct raft {
169 struct hmap_node hmap_node; /* In 'all_rafts'. */
170 struct ovsdb_log *log;
171
172/* Persistent derived state.
173 *
174 * This must be updated on stable storage before responding to RPCs. It can be
175 * derived from the header, snapshot, and log in 'log'. */
176
177 struct uuid cid; /* Cluster ID (immutable for the cluster). */
178 struct uuid sid; /* Server ID (immutable for the server). */
179 char *local_address; /* Local address (immutable for the server). */
180 char *local_nickname; /* Used for local server in log messages. */
181 char *name; /* Schema name (immutable for the cluster). */
182
183 /* Contains "struct raft_server"s and represents the server configuration
184 * most recently added to 'log'. */
185 struct hmap servers;
186
187/* Persistent state on all servers.
188 *
189 * Must be updated on stable storage before responding to RPCs. */
190
191 /* Current term and the vote for that term. These might be on the way to
192 * disk now. */
193 uint64_t term; /* Initialized to 0 and only increases. */
194 struct uuid vote; /* All-zeros if no vote yet in 'term'. */
195
196 /* The term and vote that have been synced to disk. */
197 uint64_t synced_term;
198 struct uuid synced_vote;
199
200 /* The log.
201 *
202 * A log entry with index 1 never really exists; the initial snapshot for a
203 * Raft is considered to include this index. The first real log entry has
204 * index 2.
205 *
206 * A new Raft instance contains an empty log: log_start=2, log_end=2.
207 * Over time, the log grows: log_start=2, log_end=N.
208 * At some point, the server takes a snapshot: log_start=N, log_end=N.
209 * The log continues to grow: log_start=N, log_end=N+1...
210 *
211 * Must be updated on stable storage before responding to RPCs. */
212 struct raft_entry *entries; /* Log entry i is in log[i - log_start]. */
213 uint64_t log_start; /* Index of first entry in log. */
214 uint64_t log_end; /* Index of last entry in log, plus 1. */
215 uint64_t log_synced; /* Index of last synced entry. */
216 size_t allocated_log; /* Allocated entries in 'log'. */
217
218 /* Snapshot state (see Figure 5.1)
219 *
220 * This is the state of the cluster as of the last discarded log entry,
221 * that is, at log index 'log_start - 1' (called prevIndex in Figure 5.1).
222 * Only committed log entries can be included in a snapshot. */
223 struct raft_entry snap;
224
225/* Volatile state.
226 *
227 * The snapshot is always committed, but the rest of the log might not be yet.
228 * 'last_applied' tracks what entries have been passed to the client. If the
229 * client hasn't yet read the latest snapshot, then even the snapshot isn't
230 * applied yet. Thus, the invariants are different for these members:
231 *
232 * log_start - 2 <= last_applied <= commit_index < log_end.
233 * log_start - 1 <= commit_index < log_end.
234 */
235
236 enum raft_role role; /* Current role. */
237 uint64_t commit_index; /* Max log index known to be committed. */
238 uint64_t last_applied; /* Max log index applied to state machine. */
239 struct uuid leader_sid; /* Server ID of leader (zero, if unknown). */
240
241 /* Followers and candidates only. */
242#define ELECTION_BASE_MSEC 1024
243#define ELECTION_RANGE_MSEC 1024
244 long long int election_base; /* Time of last heartbeat from leader. */
245 long long int election_timeout; /* Time at which we start an election. */
246
247 /* Used for joining a cluster. */
248 bool joining; /* Attempting to join the cluster? */
249 struct sset remote_addresses; /* Addresses to try to find other servers. */
250 long long int join_timeout; /* Time to re-send add server request. */
251
252 /* Used for leaving a cluster. */
253 bool leaving; /* True if we are leaving the cluster. */
254 bool left; /* True if we have finished leaving. */
255 long long int leave_timeout; /* Time to re-send remove server request. */
256
257 /* Failure. */
258 bool failed; /* True if unrecoverable error has occurred. */
259
260 /* File synchronization. */
261 struct ovs_list waiters; /* Contains "struct raft_waiter"s. */
262
263 /* Network connections. */
264 struct pstream *listener; /* For connections from other Raft servers. */
265 long long int listen_backoff; /* For retrying creating 'listener'. */
266 struct ovs_list conns; /* Contains struct raft_conns. */
267
268 /* Leaders only. Reinitialized after becoming leader. */
269 struct hmap add_servers; /* Contains "struct raft_server"s to add. */
270 struct raft_server *remove_server; /* Server being removed. */
271 struct hmap commands; /* Contains "struct raft_command"s. */
272#define PING_TIME_MSEC (ELECTION_BASE_MSEC / 3)
273 long long int ping_timeout; /* Time at which to send a heartbeat */
274
275 /* Candidates only. Reinitialized at start of election. */
276 int n_votes; /* Number of votes for me. */
277};
278
279/* All Raft structures. */
280static struct hmap all_rafts = HMAP_INITIALIZER(&all_rafts);
281
282static void raft_init(void);
283
284static struct ovsdb_error *raft_read_header(struct raft *)
285 OVS_WARN_UNUSED_RESULT;
286
287static void raft_send_execute_command_reply(struct raft *,
288 const struct uuid *sid,
289 const struct uuid *eid,
290 enum raft_command_status,
291 uint64_t commit_index);
292
293static void raft_update_our_match_index(struct raft *, uint64_t min_index);
294
295static void raft_send_remove_server_reply__(
296 struct raft *, const struct uuid *target_sid,
297 const struct uuid *requester_sid, struct unixctl_conn *requester_conn,
298 bool success, const char *comment);
17bd4149 299static void raft_finished_leaving_cluster(struct raft *);
1b1d2e6d
BP
300
301static void raft_server_init_leader(struct raft *, struct raft_server *);
302
303static bool raft_rpc_is_heartbeat(const union raft_rpc *);
304static bool raft_is_rpc_synced(const struct raft *, const union raft_rpc *);
305
306static void raft_handle_rpc(struct raft *, const union raft_rpc *);
17bd4149 307
02acb41a
BP
308static bool raft_send_at(struct raft *, const union raft_rpc *,
309 int line_number);
310#define raft_send(raft, rpc) raft_send_at(raft, rpc, __LINE__)
311
312static bool raft_send_to_conn_at(struct raft *, const union raft_rpc *,
313 struct raft_conn *, int line_number);
314#define raft_send_to_conn(raft, rpc, conn) \
315 raft_send_to_conn_at(raft, rpc, conn, __LINE__)
316
1b1d2e6d
BP
317static void raft_send_append_request(struct raft *,
318 struct raft_server *, unsigned int n,
319 const char *comment);
320
321static void raft_become_leader(struct raft *);
322static void raft_become_follower(struct raft *);
323static void raft_reset_timer(struct raft *);
324static void raft_send_heartbeats(struct raft *);
325static void raft_start_election(struct raft *, bool leadership_transfer);
326static bool raft_truncate(struct raft *, uint64_t new_end);
327static void raft_get_servers_from_log(struct raft *, enum vlog_level);
328
329static bool raft_handle_write_error(struct raft *, struct ovsdb_error *);
330
331static void raft_run_reconfigure(struct raft *);
332
333static struct raft_server *
334raft_find_server(const struct raft *raft, const struct uuid *sid)
335{
336 return raft_server_find(&raft->servers, sid);
337}
338
339static char *
340raft_make_address_passive(const char *address_)
341{
342 if (!strncmp(address_, "unix:", 5)) {
343 return xasprintf("p%s", address_);
344 } else {
345 char *address = xstrdup(address_);
0b043300
BP
346 char *host, *port;
347 inet_parse_host_port_tokens(strchr(address, ':') + 1, &host, &port);
1b1d2e6d
BP
348
349 struct ds paddr = DS_EMPTY_INITIALIZER;
350 ds_put_format(&paddr, "p%.3s:%s:", address, port);
351 if (strchr(host, ':')) {
352 ds_put_format(&paddr, "[%s]", host);
353 } else {
354 ds_put_cstr(&paddr, host);
355 }
356 free(address);
357 return ds_steal_cstr(&paddr);
358 }
359}
360
361static struct raft *
362raft_alloc(void)
363{
364 raft_init();
365
366 struct raft *raft = xzalloc(sizeof *raft);
367 hmap_node_nullify(&raft->hmap_node);
368 hmap_init(&raft->servers);
369 raft->log_start = raft->log_end = 1;
370 raft->role = RAFT_FOLLOWER;
371 sset_init(&raft->remote_addresses);
372 raft->join_timeout = LLONG_MAX;
373 ovs_list_init(&raft->waiters);
374 raft->listen_backoff = LLONG_MIN;
375 ovs_list_init(&raft->conns);
376 hmap_init(&raft->add_servers);
377 hmap_init(&raft->commands);
378
379 raft->ping_timeout = time_msec() + PING_TIME_MSEC;
380 raft_reset_timer(raft);
381
382 return raft;
383}
384
385/* Creates an on-disk file that represents a new Raft cluster and initializes
386 * it to consist of a single server, the one on which this function is called.
387 *
388 * Creates the local copy of the cluster's log in 'file_name', which must not
389 * already exist. Gives it the name 'name', which should be the database
390 * schema name and which is used only to match up this database with the server
391 * added to the cluster later if the cluster ID is unavailable.
392 *
393 * The new server is located at 'local_address', which must take one of the
394 * forms "tcp:IP:PORT" or "ssl:IP:PORT", where IP is an IPv4 address or a
395 * square bracket enclosed IPv6 address and PORT is a TCP port number.
396 *
397 * This only creates the on-disk file. Use raft_open() to start operating the
398 * new server.
399 *
400 * Returns null if successful, otherwise an ovsdb_error describing the
401 * problem. */
402struct ovsdb_error * OVS_WARN_UNUSED_RESULT
403raft_create_cluster(const char *file_name, const char *name,
404 const char *local_address, const struct json *data)
405{
406 /* Parse and verify validity of the local address. */
407 struct ovsdb_error *error = raft_address_validate(local_address);
408 if (error) {
409 return error;
410 }
411
412 /* Create log file. */
413 struct ovsdb_log *log;
414 error = ovsdb_log_open(file_name, RAFT_MAGIC, OVSDB_LOG_CREATE_EXCL,
415 -1, &log);
416 if (error) {
417 return error;
418 }
419
420 /* Write log file. */
421 struct raft_header h = {
422 .sid = uuid_random(),
423 .cid = uuid_random(),
424 .name = xstrdup(name),
425 .local_address = xstrdup(local_address),
426 .joining = false,
427 .remote_addresses = SSET_INITIALIZER(&h.remote_addresses),
428 .snap_index = 1,
429 .snap = {
430 .term = 1,
431 .data = json_nullable_clone(data),
432 .eid = uuid_random(),
433 .servers = json_object_create(),
434 },
435 };
436 shash_add_nocopy(json_object(h.snap.servers),
437 xasprintf(UUID_FMT, UUID_ARGS(&h.sid)),
438 json_string_create(local_address));
439 error = ovsdb_log_write_and_free(log, raft_header_to_json(&h));
440 raft_header_uninit(&h);
441 if (!error) {
442 error = ovsdb_log_commit_block(log);
443 }
444 ovsdb_log_close(log);
445
446 return error;
447}
448
449/* Creates a database file that represents a new server in an existing Raft
450 * cluster.
451 *
452 * Creates the local copy of the cluster's log in 'file_name', which must not
453 * already exist. Gives it the name 'name', which must be the same name
454 * passed in to raft_create_cluster() earlier.
455 *
456 * 'cid' is optional. If specified, the new server will join only the cluster
457 * with the given cluster ID.
458 *
459 * The new server is located at 'local_address', which must take one of the
460 * forms "tcp:IP:PORT" or "ssl:IP:PORT", where IP is an IPv4 address or a
461 * square bracket enclosed IPv6 address and PORT is a TCP port number.
462 *
463 * Joining the cluster requires contacting it. Thus, 'remote_addresses'
464 * specifies the addresses of existing servers in the cluster. One server out
465 * of the existing cluster is sufficient, as long as that server is reachable
466 * and not partitioned from the current cluster leader. If multiple servers
467 * from the cluster are specified, then it is sufficient for any of them to
468 * meet this criterion.
469 *
470 * This only creates the on-disk file and does no network access. Use
471 * raft_open() to start operating the new server. (Until this happens, the
472 * new server has not joined the cluster.)
473 *
474 * Returns null if successful, otherwise an ovsdb_error describing the
475 * problem. */
476struct ovsdb_error * OVS_WARN_UNUSED_RESULT
477raft_join_cluster(const char *file_name,
478 const char *name, const char *local_address,
479 const struct sset *remote_addresses,
480 const struct uuid *cid)
481{
482 ovs_assert(!sset_is_empty(remote_addresses));
483
484 /* Parse and verify validity of the addresses. */
485 struct ovsdb_error *error = raft_address_validate(local_address);
486 if (error) {
487 return error;
488 }
489 const char *addr;
490 SSET_FOR_EACH (addr, remote_addresses) {
491 error = raft_address_validate(addr);
492 if (error) {
493 return error;
494 }
495 if (!strcmp(addr, local_address)) {
496 return ovsdb_error(NULL, "remote addresses cannot be the same "
497 "as the local address");
498 }
499 }
500
501 /* Verify validity of the cluster ID (if provided). */
502 if (cid && uuid_is_zero(cid)) {
503 return ovsdb_error(NULL, "all-zero UUID is not valid cluster ID");
504 }
505
506 /* Create log file. */
507 struct ovsdb_log *log;
508 error = ovsdb_log_open(file_name, RAFT_MAGIC, OVSDB_LOG_CREATE_EXCL,
509 -1, &log);
510 if (error) {
511 return error;
512 }
513
514 /* Write log file. */
515 struct raft_header h = {
516 .sid = uuid_random(),
517 .cid = cid ? *cid : UUID_ZERO,
518 .name = xstrdup(name),
519 .local_address = xstrdup(local_address),
520 .joining = true,
521 /* No snapshot yet. */
522 };
523 sset_clone(&h.remote_addresses, remote_addresses);
524 error = ovsdb_log_write_and_free(log, raft_header_to_json(&h));
525 raft_header_uninit(&h);
526 if (!error) {
527 error = ovsdb_log_commit_block(log);
528 }
529 ovsdb_log_close(log);
530
531 return error;
532}
533
534/* Reads the initial header record from 'log', which must be a Raft clustered
535 * database log, and populates '*md' with the information read from it. The
536 * caller must eventually destroy 'md' with raft_metadata_destroy().
537 *
538 * On success, returns NULL. On failure, returns an error that the caller must
539 * eventually destroy and zeros '*md'. */
540struct ovsdb_error * OVS_WARN_UNUSED_RESULT
541raft_read_metadata(struct ovsdb_log *log, struct raft_metadata *md)
542{
543 struct raft *raft = raft_alloc();
544 raft->log = log;
545
546 struct ovsdb_error *error = raft_read_header(raft);
547 if (!error) {
548 md->sid = raft->sid;
549 md->name = xstrdup(raft->name);
550 md->local = xstrdup(raft->local_address);
551 md->cid = raft->cid;
552 } else {
553 memset(md, 0, sizeof *md);
554 }
555
556 raft->log = NULL;
557 raft_close(raft);
558 return error;
559}
560
561/* Frees the metadata in 'md'. */
562void
563raft_metadata_destroy(struct raft_metadata *md)
564{
565 if (md) {
566 free(md->name);
567 free(md->local);
568 }
569}
570
571static const struct raft_entry *
572raft_get_entry(const struct raft *raft, uint64_t index)
573{
574 ovs_assert(index >= raft->log_start);
575 ovs_assert(index < raft->log_end);
576 return &raft->entries[index - raft->log_start];
577}
578
579static uint64_t
580raft_get_term(const struct raft *raft, uint64_t index)
581{
582 return (index == raft->log_start - 1
583 ? raft->snap.term
584 : raft_get_entry(raft, index)->term);
585}
586
587static const struct json *
588raft_servers_for_index(const struct raft *raft, uint64_t index)
589{
590 ovs_assert(index >= raft->log_start - 1);
591 ovs_assert(index < raft->log_end);
592
593 const struct json *servers = raft->snap.servers;
594 for (uint64_t i = raft->log_start; i <= index; i++) {
595 const struct raft_entry *e = raft_get_entry(raft, i);
596 if (e->servers) {
597 servers = e->servers;
598 }
599 }
600 return servers;
601}
602
603static void
604raft_set_servers(struct raft *raft, const struct hmap *new_servers,
605 enum vlog_level level)
606{
607 struct raft_server *s, *next;
608 HMAP_FOR_EACH_SAFE (s, next, hmap_node, &raft->servers) {
609 if (!raft_server_find(new_servers, &s->sid)) {
610 ovs_assert(s != raft->remove_server);
611
612 hmap_remove(&raft->servers, &s->hmap_node);
613 VLOG(level, "server %s removed from configuration", s->nickname);
614 raft_server_destroy(s);
615 }
616 }
617
618 HMAP_FOR_EACH_SAFE (s, next, hmap_node, new_servers) {
619 if (!raft_find_server(raft, &s->sid)) {
620 VLOG(level, "server %s added to configuration", s->nickname);
621
622 struct raft_server *new
623 = raft_server_add(&raft->servers, &s->sid, s->address);
624 raft_server_init_leader(raft, new);
625 }
626 }
627}
628
629static uint64_t
630raft_add_entry(struct raft *raft,
631 uint64_t term, struct json *data, const struct uuid *eid,
632 struct json *servers)
633{
634 if (raft->log_end - raft->log_start >= raft->allocated_log) {
635 raft->entries = x2nrealloc(raft->entries, &raft->allocated_log,
636 sizeof *raft->entries);
637 }
638
639 uint64_t index = raft->log_end++;
640 struct raft_entry *entry = &raft->entries[index - raft->log_start];
641 entry->term = term;
642 entry->data = data;
643 entry->eid = eid ? *eid : UUID_ZERO;
644 entry->servers = servers;
645 return index;
646}
647
648/* Writes a RAFT_REC_ENTRY record for 'term', 'data', 'eid', 'servers' to
649 * 'raft''s log and returns an error indication. */
650static struct ovsdb_error * OVS_WARN_UNUSED_RESULT
651raft_write_entry(struct raft *raft, uint64_t term, struct json *data,
652 const struct uuid *eid, struct json *servers)
653{
654 struct raft_record r = {
655 .type = RAFT_REC_ENTRY,
656 .term = term,
657 .entry = {
658 .index = raft_add_entry(raft, term, data, eid, servers),
659 .data = data,
660 .servers = servers,
661 .eid = eid ? *eid : UUID_ZERO,
662 },
663 };
664 return ovsdb_log_write_and_free(raft->log, raft_record_to_json(&r));
665}
666
667static struct ovsdb_error * OVS_WARN_UNUSED_RESULT
668raft_write_state(struct ovsdb_log *log,
669 uint64_t term, const struct uuid *vote)
670{
671 struct raft_record r = { .term = term };
672 if (vote && !uuid_is_zero(vote)) {
673 r.type = RAFT_REC_VOTE;
674 r.sid = *vote;
675 } else {
676 r.type = RAFT_REC_TERM;
677 }
678 return ovsdb_log_write_and_free(log, raft_record_to_json(&r));
679}
680
681static struct ovsdb_error * OVS_WARN_UNUSED_RESULT
682raft_apply_record(struct raft *raft, unsigned long long int rec_idx,
683 const struct raft_record *r)
684{
685 /* Apply "term", which is present in most kinds of records (and otherwise
686 * 0).
687 *
688 * A Raft leader can replicate entries from previous terms to the other
689 * servers in the cluster, retaining the original terms on those entries
690 * (see section 3.6.2 "Committing entries from previous terms" for more
691 * information), so it's OK for the term in a log record to precede the
692 * current term. */
693 if (r->term > raft->term) {
694 raft->term = raft->synced_term = r->term;
695 raft->vote = raft->synced_vote = UUID_ZERO;
696 }
697
698 switch (r->type) {
699 case RAFT_REC_ENTRY:
700 if (r->entry.index < raft->commit_index) {
701 return ovsdb_error(NULL, "record %llu attempts to truncate log "
702 "from %"PRIu64" to %"PRIu64" entries, but "
703 "commit index is already %"PRIu64,
704 rec_idx, raft->log_end, r->entry.index,
705 raft->commit_index);
706 } else if (r->entry.index > raft->log_end) {
707 return ovsdb_error(NULL, "record %llu with index %"PRIu64" skips "
708 "past expected index %"PRIu64,
709 rec_idx, r->entry.index, raft->log_end);
710 }
711
712 if (r->entry.index < raft->log_end) {
713 /* This can happen, but it is notable. */
714 VLOG_DBG("record %llu truncates log from %"PRIu64" to %"PRIu64
715 " entries", rec_idx, raft->log_end, r->entry.index);
716 raft_truncate(raft, r->entry.index);
717 }
718
719 uint64_t prev_term = (raft->log_end > raft->log_start
720 ? raft->entries[raft->log_end
721 - raft->log_start - 1].term
722 : raft->snap.term);
723 if (r->term < prev_term) {
724 return ovsdb_error(NULL, "record %llu with index %"PRIu64" term "
725 "%"PRIu64" precedes previous entry's term "
726 "%"PRIu64,
727 rec_idx, r->entry.index, r->term, prev_term);
728 }
729
730 raft->log_synced = raft_add_entry(
731 raft, r->term,
732 json_nullable_clone(r->entry.data), &r->entry.eid,
733 json_nullable_clone(r->entry.servers));
734 return NULL;
735
736 case RAFT_REC_TERM:
737 return NULL;
738
739 case RAFT_REC_VOTE:
740 if (r->term < raft->term) {
741 return ovsdb_error(NULL, "record %llu votes for term %"PRIu64" "
742 "but current term is %"PRIu64,
743 rec_idx, r->term, raft->term);
744 } else if (!uuid_is_zero(&raft->vote)
745 && !uuid_equals(&raft->vote, &r->sid)) {
746 return ovsdb_error(NULL, "record %llu votes for "SID_FMT" in term "
747 "%"PRIu64" but a previous record for the "
748 "same term voted for "SID_FMT, rec_idx,
749 SID_ARGS(&raft->vote), r->term,
750 SID_ARGS(&r->sid));
751 } else {
752 raft->vote = raft->synced_vote = r->sid;
753 return NULL;
754 }
755 break;
756
757 case RAFT_REC_NOTE:
758 if (!strcmp(r->note, "left")) {
759 return ovsdb_error(NULL, "record %llu indicates server has left "
760 "the cluster; it cannot be added back (use "
761 "\"ovsdb-tool join-cluster\" to add a new "
762 "server)", rec_idx);
763 }
764 return NULL;
765
766 case RAFT_REC_COMMIT_INDEX:
767 if (r->commit_index < raft->commit_index) {
768 return ovsdb_error(NULL, "record %llu regresses commit index "
769 "from %"PRIu64 " to %"PRIu64,
770 rec_idx, raft->commit_index, r->commit_index);
771 } else if (r->commit_index >= raft->log_end) {
772 return ovsdb_error(NULL, "record %llu advances commit index to "
773 "%"PRIu64 " but last log index is %"PRIu64,
774 rec_idx, r->commit_index, raft->log_end - 1);
775 } else {
776 raft->commit_index = r->commit_index;
777 return NULL;
778 }
779 break;
780
781 case RAFT_REC_LEADER:
782 /* XXX we could use this to take back leadership for quick restart */
783 return NULL;
784
785 default:
786 OVS_NOT_REACHED();
787 }
788}
789
790static struct ovsdb_error * OVS_WARN_UNUSED_RESULT
791raft_read_header(struct raft *raft)
792{
793 /* Read header record. */
794 struct json *json;
795 struct ovsdb_error *error = ovsdb_log_read(raft->log, &json);
796 if (error || !json) {
797 /* Report error or end-of-file. */
798 return error;
799 }
800 ovsdb_log_mark_base(raft->log);
801
802 struct raft_header h;
803 error = raft_header_from_json(&h, json);
804 json_destroy(json);
805 if (error) {
806 return error;
807 }
808
809 raft->sid = h.sid;
810 raft->cid = h.cid;
811 raft->name = xstrdup(h.name);
812 raft->local_address = xstrdup(h.local_address);
813 raft->local_nickname = raft_address_to_nickname(h.local_address, &h.sid);
814 raft->joining = h.joining;
815
816 if (h.joining) {
817 sset_clone(&raft->remote_addresses, &h.remote_addresses);
818 } else {
819 raft_entry_clone(&raft->snap, &h.snap);
820 raft->log_start = raft->log_end = h.snap_index + 1;
821 raft->commit_index = h.snap_index;
822 raft->last_applied = h.snap_index - 1;
823 }
824
825 raft_header_uninit(&h);
826
827 return NULL;
828}
829
830static struct ovsdb_error * OVS_WARN_UNUSED_RESULT
831raft_read_log(struct raft *raft)
832{
833 for (unsigned long long int i = 1; ; i++) {
834 struct json *json;
835 struct ovsdb_error *error = ovsdb_log_read(raft->log, &json);
836 if (!json) {
837 if (error) {
838 /* We assume that the error is due to a partial write while
839 * appending to the file before a crash, so log it and
840 * continue. */
841 char *error_string = ovsdb_error_to_string_free(error);
842 VLOG_WARN("%s", error_string);
843 free(error_string);
844 error = NULL;
845 }
846 break;
847 }
848
849 struct raft_record r;
850 error = raft_record_from_json(&r, json);
851 if (!error) {
852 error = raft_apply_record(raft, i, &r);
853 raft_record_uninit(&r);
854 }
855 if (error) {
856 return ovsdb_wrap_error(error, "error reading record %llu from "
857 "%s log", i, raft->name);
858 }
859 }
860
861 /* Set the most recent servers. */
862 raft_get_servers_from_log(raft, VLL_DBG);
863
864 return NULL;
865}
866
867static void
868raft_reset_timer(struct raft *raft)
869{
870 unsigned int duration = (ELECTION_BASE_MSEC
871 + random_range(ELECTION_RANGE_MSEC));
872 raft->election_base = time_msec();
873 raft->election_timeout = raft->election_base + duration;
874}
875
876static void
877raft_add_conn(struct raft *raft, struct jsonrpc_session *js,
878 const struct uuid *sid, bool incoming)
879{
880 struct raft_conn *conn = xzalloc(sizeof *conn);
881 ovs_list_push_back(&raft->conns, &conn->list_node);
882 conn->js = js;
883 if (sid) {
884 conn->sid = *sid;
885 }
886 conn->nickname = raft_address_to_nickname(jsonrpc_session_get_name(js),
887 &conn->sid);
888 conn->incoming = incoming;
889 conn->js_seqno = jsonrpc_session_get_seqno(conn->js);
890}
891
892/* Starts the local server in an existing Raft cluster, using the local copy of
893 * the cluster's log in 'file_name'. Takes ownership of 'log', whether
894 * successful or not. */
895struct ovsdb_error * OVS_WARN_UNUSED_RESULT
896raft_open(struct ovsdb_log *log, struct raft **raftp)
897{
898 struct raft *raft = raft_alloc();
899 raft->log = log;
900
901 struct ovsdb_error *error = raft_read_header(raft);
902 if (error) {
903 goto error;
904 }
905
906 if (!raft->joining) {
907 error = raft_read_log(raft);
908 if (error) {
909 goto error;
910 }
911
912 /* Find our own server. */
913 if (!raft_find_server(raft, &raft->sid)) {
914 error = ovsdb_error(NULL, "server does not belong to cluster");
915 goto error;
916 }
917
918 /* If there's only one server, start an election right away so that the
919 * cluster bootstraps quickly. */
920 if (hmap_count(&raft->servers) == 1) {
921 raft_start_election(raft, false);
922 }
923 } else {
924 raft->join_timeout = time_msec() + 1000;
925 }
926
927 *raftp = raft;
928 hmap_insert(&all_rafts, &raft->hmap_node, hash_string(raft->name, 0));
929 return NULL;
930
931error:
932 raft_close(raft);
933 *raftp = NULL;
934 return error;
935}
936
937/* Returns the name of 'raft', which in OVSDB is the database schema name. */
938const char *
939raft_get_name(const struct raft *raft)
940{
941 return raft->name;
942}
943
944/* Returns the cluster ID of 'raft'. If 'raft' has not yet completed joining
945 * its cluster, then 'cid' will be all-zeros (unless the administrator
946 * specified a cluster ID running "ovsdb-tool join-cluster").
947 *
948 * Each cluster has a unique cluster ID. */
949const struct uuid *
950raft_get_cid(const struct raft *raft)
951{
952 return &raft->cid;
953}
954
955/* Returns the server ID of 'raft'. Each server has a unique server ID. */
956const struct uuid *
957raft_get_sid(const struct raft *raft)
958{
959 return &raft->sid;
960}
961
962/* Returns true if 'raft' has completed joining its cluster, has not left or
963 * initiated leaving the cluster, does not have failed disk storage, and is
964 * apparently connected to the leader in a healthy way (or is itself the
965 * leader).*/
966bool
967raft_is_connected(const struct raft *raft)
968{
969 return (raft->role != RAFT_CANDIDATE
970 && !raft->joining
971 && !raft->leaving
972 && !raft->left
973 && !raft->failed);
974}
975
976/* Returns true if 'raft' is the cluster leader. */
977bool
978raft_is_leader(const struct raft *raft)
979{
980 return raft->role == RAFT_LEADER;
981}
982
983/* Returns true if 'raft' is the process of joining its cluster. */
984bool
985raft_is_joining(const struct raft *raft)
986{
987 return raft->joining;
988}
989
990/* Only returns *connected* connections. */
991static struct raft_conn *
992raft_find_conn_by_sid(struct raft *raft, const struct uuid *sid)
993{
994 if (!uuid_is_zero(sid)) {
995 struct raft_conn *conn;
996 LIST_FOR_EACH (conn, list_node, &raft->conns) {
997 if (uuid_equals(sid, &conn->sid)
998 && jsonrpc_session_is_connected(conn->js)) {
999 return conn;
1000 }
1001 }
1002 }
1003 return NULL;
1004}
1005
1006static struct raft_conn *
1007raft_find_conn_by_address(struct raft *raft, const char *address)
1008{
1009 struct raft_conn *conn;
1010 LIST_FOR_EACH (conn, list_node, &raft->conns) {
1011 if (!strcmp(jsonrpc_session_get_name(conn->js), address)) {
1012 return conn;
1013 }
1014 }
1015 return NULL;
1016}
1017
1018static void OVS_PRINTF_FORMAT(3, 4)
1019raft_record_note(struct raft *raft, const char *note,
1020 const char *comment_format, ...)
1021{
1022 va_list args;
1023 va_start(args, comment_format);
1024 char *comment = xvasprintf(comment_format, args);
1025 va_end(args);
1026
1027 struct raft_record r = {
1028 .type = RAFT_REC_NOTE,
1029 .comment = comment,
1030 .note = CONST_CAST(char *, note),
1031 };
1032 ignore(ovsdb_log_write_and_free(raft->log, raft_record_to_json(&r)));
1033
1034 free(comment);
1035}
1036
1037/* If we're leader, try to transfer leadership to another server, logging
1038 * 'reason' as the human-readable reason (it should be a phrase suitable for
1039 * following "because") . */
1040void
1041raft_transfer_leadership(struct raft *raft, const char *reason)
1042{
1043 if (raft->role != RAFT_LEADER) {
1044 return;
1045 }
1046
1047 struct raft_server *s;
1048 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
1049 if (!uuid_equals(&raft->sid, &s->sid)
1050 && s->phase == RAFT_PHASE_STABLE) {
1051 struct raft_conn *conn = raft_find_conn_by_sid(raft, &s->sid);
1052 if (!conn) {
1053 continue;
1054 }
1055
1056 union raft_rpc rpc = {
1057 .become_leader = {
1058 .common = {
1059 .comment = CONST_CAST(char *, reason),
1060 .type = RAFT_RPC_BECOME_LEADER,
1061 .sid = s->sid,
1062 },
1063 .term = raft->term,
1064 }
1065 };
02acb41a 1066 raft_send_to_conn(raft, &rpc, conn);
1b1d2e6d
BP
1067
1068 raft_record_note(raft, "transfer leadership",
1069 "transferring leadership to %s because %s",
1070 s->nickname, reason);
1071 break;
1072 }
1073 }
1074}
1075
1076/* Send a RemoveServerRequest to the rest of the servers in the cluster.
1077 *
1078 * If we know which server is the leader, we can just send the request to it.
1079 * However, we might not know which server is the leader, and we might never
1080 * find out if the remove request was actually previously committed by a
1081 * majority of the servers (because in that case the new leader will not send
1082 * AppendRequests or heartbeats to us). Therefore, we instead send
1083 * RemoveRequests to every server. This theoretically has the same problem, if
1084 * the current cluster leader was not previously a member of the cluster, but
1085 * it seems likely to be more robust in practice. */
1086static void
1087raft_send_remove_server_requests(struct raft *raft)
1088{
1089 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
1090 VLOG_INFO_RL(&rl, "sending remove request (joining=%s, leaving=%s)",
1091 raft->joining ? "true" : "false",
1092 raft->leaving ? "true" : "false");
1093 const struct raft_server *s;
1094 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
1095 if (!uuid_equals(&s->sid, &raft->sid)) {
1096 union raft_rpc rpc = (union raft_rpc) {
1097 .remove_server_request = {
1098 .common = {
1099 .type = RAFT_RPC_REMOVE_SERVER_REQUEST,
1100 .sid = s->sid,
1101 },
1102 .sid = raft->sid,
1103 },
1104 };
1105 raft_send(raft, &rpc);
1106 }
1107 }
1108
1109 raft->leave_timeout = time_msec() + ELECTION_BASE_MSEC;
1110}
1111
1112/* Attempts to start 'raft' leaving its cluster. The caller can check progress
1113 * using raft_is_leaving() and raft_left(). */
1114void
1115raft_leave(struct raft *raft)
1116{
1117 if (raft->joining || raft->failed || raft->leaving || raft->left) {
1118 return;
1119 }
1120 VLOG_INFO(SID_FMT": starting to leave cluster "CID_FMT,
1121 SID_ARGS(&raft->sid), CID_ARGS(&raft->cid));
1122 raft->leaving = true;
1123 raft_transfer_leadership(raft, "this server is leaving the cluster");
1124 raft_become_follower(raft);
1125 raft_send_remove_server_requests(raft);
1126 raft->leave_timeout = time_msec() + ELECTION_BASE_MSEC;
1127}
1128
1129/* Returns true if 'raft' is currently attempting to leave its cluster. */
1130bool
1131raft_is_leaving(const struct raft *raft)
1132{
1133 return raft->leaving;
1134}
1135
1136/* Returns true if 'raft' successfully left its cluster. */
1137bool
1138raft_left(const struct raft *raft)
1139{
1140 return raft->left;
1141}
1142
1143/* Returns true if 'raft' has experienced a disk I/O failure. When this
1144 * returns true, only closing and reopening 'raft' allows for recovery. */
1145bool
1146raft_failed(const struct raft *raft)
1147{
1148 return raft->failed;
1149}
1150
1151/* Forces 'raft' to attempt to take leadership of the cluster by deposing the
1152 * current cluster. */
1153void
1154raft_take_leadership(struct raft *raft)
1155{
1156 if (raft->role != RAFT_LEADER) {
1157 raft_start_election(raft, true);
1158 }
1159}
1160
1161/* Closes everything owned by 'raft' that might be visible outside the process:
1162 * network connections, commands, etc. This is part of closing 'raft'; it is
1163 * also used if 'raft' has failed in an unrecoverable way. */
1164static void
1165raft_close__(struct raft *raft)
1166{
1167 if (!hmap_node_is_null(&raft->hmap_node)) {
1168 hmap_remove(&all_rafts, &raft->hmap_node);
1169 hmap_node_nullify(&raft->hmap_node);
1170 }
1171
1172 raft_complete_all_commands(raft, RAFT_CMD_SHUTDOWN);
1173
1174 struct raft_server *rs = raft->remove_server;
1175 if (rs) {
1176 raft_send_remove_server_reply__(raft, &rs->sid, &rs->requester_sid,
1177 rs->requester_conn, false,
1178 RAFT_SERVER_SHUTDOWN);
1179 raft_server_destroy(raft->remove_server);
1180 raft->remove_server = NULL;
1181 }
1182
1183 struct raft_conn *conn, *next;
1184 LIST_FOR_EACH_SAFE (conn, next, list_node, &raft->conns) {
1185 raft_conn_close(conn);
1186 }
1187}
1188
1189/* Closes and frees 'raft'.
1190 *
1191 * A server's cluster membership is independent of whether the server is
1192 * actually running. When a server that is a member of a cluster closes, the
1193 * cluster treats this as a server failure. */
1194void
1195raft_close(struct raft *raft)
1196{
1197 if (!raft) {
1198 return;
1199 }
1200
1201 raft_transfer_leadership(raft, "this server is shutting down");
1202
1203 raft_close__(raft);
1204
1205 ovsdb_log_close(raft->log);
1206
1207 raft_servers_destroy(&raft->servers);
1208
1209 for (uint64_t index = raft->log_start; index < raft->log_end; index++) {
1210 struct raft_entry *e = &raft->entries[index - raft->log_start];
1211 raft_entry_uninit(e);
1212 }
1213 free(raft->entries);
1214
1215 raft_entry_uninit(&raft->snap);
1216
1217 raft_waiters_destroy(raft);
1218
1219 raft_servers_destroy(&raft->add_servers);
1220
1221 hmap_destroy(&raft->commands);
1222
1223 pstream_close(raft->listener);
1224
1225 sset_destroy(&raft->remote_addresses);
1226 free(raft->local_address);
1227 free(raft->local_nickname);
1228 free(raft->name);
1229
1230 free(raft);
1231}
1232
1233static bool
1234raft_conn_receive(struct raft *raft, struct raft_conn *conn,
1235 union raft_rpc *rpc)
1236{
1237 struct jsonrpc_msg *msg = jsonrpc_session_recv(conn->js);
1238 if (!msg) {
1239 return false;
1240 }
1241
1242 struct ovsdb_error *error = raft_rpc_from_jsonrpc(&raft->cid, &raft->sid,
1243 msg, rpc);
1244 jsonrpc_msg_destroy(msg);
1245 if (error) {
1246 char *s = ovsdb_error_to_string_free(error);
1247 VLOG_INFO("%s: %s", jsonrpc_session_get_name(conn->js), s);
1248 free(s);
1249 return false;
1250 }
1251
1252 if (uuid_is_zero(&conn->sid)) {
1253 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(50, 50);
1254 conn->sid = rpc->common.sid;
1255 VLOG_INFO_RL(&rl, "%s: learned server ID "SID_FMT,
1256 jsonrpc_session_get_name(conn->js), SID_ARGS(&conn->sid));
1257 } else if (!uuid_equals(&conn->sid, &rpc->common.sid)) {
1258 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
1259 VLOG_WARN_RL(&rl, "%s: ignoring message with unexpected server ID "
1260 SID_FMT" (expected "SID_FMT")",
1261 jsonrpc_session_get_name(conn->js),
1262 SID_ARGS(&rpc->common.sid), SID_ARGS(&conn->sid));
1263 raft_rpc_uninit(rpc);
1264 return false;
1265 }
1266
1267 const char *address = (rpc->type == RAFT_RPC_HELLO_REQUEST
1268 ? rpc->hello_request.address
1269 : rpc->type == RAFT_RPC_ADD_SERVER_REQUEST
1270 ? rpc->add_server_request.address
1271 : NULL);
1272 if (address) {
1273 char *new_nickname = raft_address_to_nickname(address, &conn->sid);
1274 if (strcmp(conn->nickname, new_nickname)) {
1275 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(50, 50);
1276 VLOG_INFO_RL(&rl, "%s: learned remote address %s",
1277 jsonrpc_session_get_name(conn->js), address);
1278
1279 free(conn->nickname);
1280 conn->nickname = new_nickname;
1281 } else {
1282 free(new_nickname);
1283 }
1284 }
1285
1286 return true;
1287}
1288
1289static const char *
1290raft_get_nickname(const struct raft *raft, const struct uuid *sid,
1291 char buf[SID_LEN + 1], size_t bufsize)
1292{
1293 if (uuid_equals(sid, &raft->sid)) {
1294 return raft->local_nickname;
1295 }
1296
1297 const char *s = raft_servers_get_nickname__(&raft->servers, sid);
1298 if (s) {
1299 return s;
1300 }
1301
1302 return raft_servers_get_nickname(&raft->add_servers, sid, buf, bufsize);
1303}
1304
1305static void
02acb41a
BP
1306log_rpc(const union raft_rpc *rpc, const char *direction,
1307 const struct raft_conn *conn, int line_number)
1b1d2e6d
BP
1308{
1309 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(600, 600);
1310 if (!raft_rpc_is_heartbeat(rpc) && !VLOG_DROP_DBG(&rl)) {
1311 struct ds s = DS_EMPTY_INITIALIZER;
02acb41a
BP
1312 if (line_number) {
1313 ds_put_format(&s, "raft.c:%d ", line_number);
1314 }
1315 ds_put_format(&s, "%s%s ", direction, conn->nickname);
1b1d2e6d 1316 raft_rpc_format(rpc, &s);
02acb41a 1317 VLOG_DBG("%s", ds_cstr(&s));
1b1d2e6d
BP
1318 ds_destroy(&s);
1319 }
1320}
1321
1322static void
1323raft_send_add_server_request(struct raft *raft, struct raft_conn *conn)
1324{
1325 union raft_rpc rq = {
1326 .add_server_request = {
1327 .common = {
1328 .type = RAFT_RPC_ADD_SERVER_REQUEST,
1329 .sid = UUID_ZERO,
1330 .comment = NULL,
1331 },
1332 .address = raft->local_address,
1333 },
1334 };
02acb41a 1335 raft_send_to_conn(raft, &rq, conn);
1b1d2e6d
BP
1336}
1337
1338static void
1339raft_conn_run(struct raft *raft, struct raft_conn *conn)
1340{
1341 jsonrpc_session_run(conn->js);
1342
1343 unsigned int new_seqno = jsonrpc_session_get_seqno(conn->js);
1344 bool just_connected = (new_seqno != conn->js_seqno
1345 && jsonrpc_session_is_connected(conn->js));
1346 conn->js_seqno = new_seqno;
1347 if (just_connected) {
1348 if (raft->joining) {
1349 raft_send_add_server_request(raft, conn);
1350 } else if (raft->leaving) {
1351 union raft_rpc rq = {
1352 .remove_server_request = {
1353 .common = {
1354 .type = RAFT_RPC_REMOVE_SERVER_REQUEST,
1355 .sid = conn->sid,
1356 },
1357 .sid = raft->sid,
1358 },
1359 };
02acb41a 1360 raft_send_to_conn(raft, &rq, conn);
1b1d2e6d
BP
1361 } else {
1362 union raft_rpc rq = (union raft_rpc) {
1363 .hello_request = {
1364 .common = {
1365 .type = RAFT_RPC_HELLO_REQUEST,
1366 .sid = conn->sid,
1367 },
1368 .address = raft->local_address,
1369 },
1370 };
02acb41a 1371 raft_send_to_conn(raft, &rq, conn);
1b1d2e6d
BP
1372 }
1373 }
1374
1375 for (size_t i = 0; i < 50; i++) {
1376 union raft_rpc rpc;
1377 if (!raft_conn_receive(raft, conn, &rpc)) {
1378 break;
1379 }
1380
02acb41a 1381 log_rpc(&rpc, "<--", conn, 0);
1b1d2e6d
BP
1382 raft_handle_rpc(raft, &rpc);
1383 raft_rpc_uninit(&rpc);
1384 }
1385}
1386
1387static void
1388raft_waiter_complete_rpc(struct raft *raft, const union raft_rpc *rpc)
1389{
1390 uint64_t term = raft_rpc_get_term(rpc);
1391 if (term && term < raft->term) {
1392 /* Drop the message because it's for an expired term. */
1393 return;
1394 }
1395
1396 if (!raft_is_rpc_synced(raft, rpc)) {
1397 /* This is a bug. A reply message is deferred because some state in
1398 * the message, such as a term or index, has not been committed to
1399 * disk, and they should only be completed when that commit is done.
1400 * But this message is being completed before the commit is finished.
1401 * Complain, and hope that someone reports the bug. */
1402 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
1403 if (VLOG_DROP_ERR(&rl)) {
1404 return;
1405 }
1406
1407 struct ds s = DS_EMPTY_INITIALIZER;
1408
1409 if (term > raft->synced_term) {
1410 ds_put_format(&s, " because message term %"PRIu64" is "
1411 "past synced term %"PRIu64,
1412 term, raft->synced_term);
1413 }
1414
1415 uint64_t index = raft_rpc_get_min_sync_index(rpc);
1416 if (index > raft->log_synced) {
1417 ds_put_format(&s, " %s message index %"PRIu64" is past last "
1418 "synced index %"PRIu64,
1419 s.length ? "and" : "because",
1420 index, raft->log_synced);
1421 }
1422
1423 const struct uuid *vote = raft_rpc_get_vote(rpc);
1424 if (vote && !uuid_equals(vote, &raft->synced_vote)) {
1425 char buf1[SID_LEN + 1];
1426 char buf2[SID_LEN + 1];
1427 ds_put_format(&s, " %s vote %s differs from synced vote %s",
1428 s.length ? "and" : "because",
1429 raft_get_nickname(raft, vote, buf1, sizeof buf1),
1430 raft_get_nickname(raft, &raft->synced_vote,
1431 buf2, sizeof buf2));
1432 }
1433
1434 char buf[SID_LEN + 1];
1435 ds_put_format(&s, ": %s ",
1436 raft_get_nickname(raft, &rpc->common.sid,
1437 buf, sizeof buf));
1438 raft_rpc_format(rpc, &s);
1439 VLOG_ERR("internal error: deferred %s message completed "
1440 "but not ready to send%s",
1441 raft_rpc_type_to_string(rpc->type), ds_cstr(&s));
1442 ds_destroy(&s);
1443
1444 return;
1445 }
1446
1447 struct raft_conn *dst = raft_find_conn_by_sid(raft, &rpc->common.sid);
1448 if (dst) {
02acb41a 1449 raft_send_to_conn(raft, rpc, dst);
1b1d2e6d
BP
1450 }
1451}
1452
1453static void
1454raft_waiter_complete(struct raft *raft, struct raft_waiter *w)
1455{
1456 switch (w->type) {
1457 case RAFT_W_ENTRY:
1458 if (raft->role == RAFT_LEADER) {
1459 raft_update_our_match_index(raft, w->entry.index);
1460 }
1461 raft->log_synced = w->entry.index;
1462 break;
1463
1464 case RAFT_W_TERM:
1465 raft->synced_term = w->term.term;
1466 raft->synced_vote = w->term.vote;
1467 break;
1468
1469 case RAFT_W_RPC:
1470 raft_waiter_complete_rpc(raft, w->rpc);
1471 break;
1472 }
1473}
1474
1475static void
1476raft_waiter_destroy(struct raft_waiter *w)
1477{
1478 if (!w) {
1479 return;
1480 }
1481
1482 ovs_list_remove(&w->list_node);
1483
1484 switch (w->type) {
1485 case RAFT_W_ENTRY:
1486 case RAFT_W_TERM:
1487 break;
1488
1489 case RAFT_W_RPC:
1490 raft_rpc_uninit(w->rpc);
1491 free(w->rpc);
1492 break;
1493 }
1494 free(w);
1495}
1496
1497static void
1498raft_waiters_run(struct raft *raft)
1499{
1500 if (ovs_list_is_empty(&raft->waiters)) {
1501 return;
1502 }
1503
1504 uint64_t cur = ovsdb_log_commit_progress(raft->log);
1505 struct raft_waiter *w, *next;
1506 LIST_FOR_EACH_SAFE (w, next, list_node, &raft->waiters) {
1507 if (cur < w->commit_ticket) {
1508 break;
1509 }
1510 raft_waiter_complete(raft, w);
1511 raft_waiter_destroy(w);
1512 }
1513}
1514
1515static void
1516raft_waiters_wait(struct raft *raft)
1517{
1518 struct raft_waiter *w;
1519 LIST_FOR_EACH (w, list_node, &raft->waiters) {
1520 ovsdb_log_commit_wait(raft->log, w->commit_ticket);
1521 break;
1522 }
1523}
1524
1525static void
1526raft_waiters_destroy(struct raft *raft)
1527{
1528 struct raft_waiter *w, *next;
1529 LIST_FOR_EACH_SAFE (w, next, list_node, &raft->waiters) {
1530 raft_waiter_destroy(w);
1531 }
1532}
1533
1534static bool OVS_WARN_UNUSED_RESULT
1535raft_set_term(struct raft *raft, uint64_t term, const struct uuid *vote)
1536{
1537 struct ovsdb_error *error = raft_write_state(raft->log, term, vote);
1538 if (!raft_handle_write_error(raft, error)) {
1539 return false;
1540 }
1541
1542 struct raft_waiter *w = raft_waiter_create(raft, RAFT_W_TERM, true);
1543 raft->term = w->term.term = term;
1544 raft->vote = w->term.vote = vote ? *vote : UUID_ZERO;
1545 return true;
1546}
1547
1548static void
1549raft_accept_vote(struct raft *raft, struct raft_server *s,
1550 const struct uuid *vote)
1551{
1552 if (uuid_equals(&s->vote, vote)) {
1553 return;
1554 }
1555 if (!uuid_is_zero(&s->vote)) {
1556 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
1557 char buf1[SID_LEN + 1];
1558 char buf2[SID_LEN + 1];
1559 VLOG_WARN_RL(&rl, "server %s changed its vote from %s to %s",
1560 s->nickname,
1561 raft_get_nickname(raft, &s->vote, buf1, sizeof buf1),
1562 raft_get_nickname(raft, vote, buf2, sizeof buf2));
1563 }
1564 s->vote = *vote;
1565 if (uuid_equals(vote, &raft->sid)
1566 && ++raft->n_votes > hmap_count(&raft->servers) / 2) {
1567 raft_become_leader(raft);
1568 }
1569}
1570
1571static void
1572raft_start_election(struct raft *raft, bool leadership_transfer)
1573{
1574 if (raft->leaving) {
1575 return;
1576 }
1577
1578 struct raft_server *me = raft_find_server(raft, &raft->sid);
1579 if (!me) {
1580 return;
1581 }
1582
1583 if (!raft_set_term(raft, raft->term + 1, &raft->sid)) {
1584 return;
1585 }
1586
1587 raft_complete_all_commands(raft, RAFT_CMD_LOST_LEADERSHIP);
1588
1589 ovs_assert(raft->role != RAFT_LEADER);
1590 ovs_assert(hmap_is_empty(&raft->commands));
1591 raft->role = RAFT_CANDIDATE;
1592
1593 raft->n_votes = 0;
1594
1595 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
1596 if (!VLOG_DROP_INFO(&rl)) {
1597 long long int now = time_msec();
1598 if (now >= raft->election_timeout) {
1599 VLOG_INFO("term %"PRIu64": %lld ms timeout expired, "
1600 "starting election",
1601 raft->term, now - raft->election_base);
1602 } else {
1603 VLOG_INFO("term %"PRIu64": starting election", raft->term);
1604 }
1605 }
1606 raft_reset_timer(raft);
1607
1608 struct raft_server *peer;
1609 HMAP_FOR_EACH (peer, hmap_node, &raft->servers) {
1610 peer->vote = UUID_ZERO;
1611 if (uuid_equals(&raft->sid, &peer->sid)) {
1612 continue;
1613 }
1614
1615 union raft_rpc rq = {
1616 .vote_request = {
1617 .common = {
1618 .type = RAFT_RPC_VOTE_REQUEST,
1619 .sid = peer->sid,
1620 },
1621 .term = raft->term,
1622 .last_log_index = raft->log_end - 1,
1623 .last_log_term = (
1624 raft->log_end > raft->log_start
1625 ? raft->entries[raft->log_end - raft->log_start - 1].term
1626 : raft->snap.term),
1627 .leadership_transfer = leadership_transfer,
1628 },
1629 };
1630 raft_send(raft, &rq);
1631 }
1632
1633 /* Vote for ourselves. */
1634 raft_accept_vote(raft, me, &raft->sid);
1635}
1636
1637static void
1638raft_open_conn(struct raft *raft, const char *address, const struct uuid *sid)
1639{
1640 if (strcmp(address, raft->local_address)
1641 && !raft_find_conn_by_address(raft, address)) {
1642 raft_add_conn(raft, jsonrpc_session_open(address, true), sid, false);
1643 }
1644}
1645
1646static void
1647raft_conn_close(struct raft_conn *conn)
1648{
1649 jsonrpc_session_close(conn->js);
1650 ovs_list_remove(&conn->list_node);
1651 free(conn->nickname);
1652 free(conn);
1653}
1654
1655/* Returns true if 'conn' should stay open, false if it should be closed. */
1656static bool
1657raft_conn_should_stay_open(struct raft *raft, struct raft_conn *conn)
1658{
1659 /* Close the connection if it's actually dead. If necessary, we'll
1660 * initiate a new session later. */
1661 if (!jsonrpc_session_is_alive(conn->js)) {
1662 return false;
1663 }
1664
1665 /* Keep incoming sessions. We trust the originator to decide to drop
1666 * it. */
1667 if (conn->incoming) {
1668 return true;
1669 }
1670
1671 /* If we are joining the cluster, keep sessions to the remote addresses
1672 * that are supposed to be part of the cluster we're joining. */
1673 if (raft->joining && sset_contains(&raft->remote_addresses,
1674 jsonrpc_session_get_name(conn->js))) {
1675 return true;
1676 }
1677
1678 /* We have joined the cluster. If we did that "recently", then there is a
1679 * chance that we do not have the most recent server configuration log
1680 * entry. If so, it's a waste to disconnect from the servers that were in
1681 * remote_addresses and that will probably appear in the configuration,
1682 * just to reconnect to them a moment later when we do get the
1683 * configuration update. If we are not ourselves in the configuration,
1684 * then we know that there must be a new configuration coming up, so in
1685 * that case keep the connection. */
1686 if (!raft_find_server(raft, &raft->sid)) {
1687 return true;
1688 }
1689
1690 /* Keep the connection only if the server is part of the configuration. */
1691 return raft_find_server(raft, &conn->sid);
1692}
1693
1694/* Allows 'raft' to maintain the distributed log. Call this function as part
1695 * of the process's main loop. */
1696void
1697raft_run(struct raft *raft)
1698{
1699 if (raft->left || raft->failed) {
1700 return;
1701 }
1702
1703 raft_waiters_run(raft);
1704
1705 if (!raft->listener && time_msec() >= raft->listen_backoff) {
1706 char *paddr = raft_make_address_passive(raft->local_address);
1707 int error = pstream_open(paddr, &raft->listener, DSCP_DEFAULT);
1708 if (error) {
1709 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
1710 VLOG_WARN_RL(&rl, "%s: listen failed (%s)",
1711 paddr, ovs_strerror(error));
1712 raft->listen_backoff = time_msec() + 1000;
1713 }
1714 free(paddr);
1715 }
1716
1717 if (raft->listener) {
1718 struct stream *stream;
1719 int error = pstream_accept(raft->listener, &stream);
1720 if (!error) {
1721 raft_add_conn(raft, jsonrpc_session_open_unreliably(
1722 jsonrpc_open(stream), DSCP_DEFAULT), NULL,
1723 true);
1724 } else if (error != EAGAIN) {
1725 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
1726 VLOG_WARN_RL(&rl, "%s: accept failed: %s",
1727 pstream_get_name(raft->listener),
1728 ovs_strerror(error));
1729 }
1730 }
1731
1732 /* Run RPCs for all open sessions. */
1733 struct raft_conn *conn;
1734 LIST_FOR_EACH (conn, list_node, &raft->conns) {
1735 raft_conn_run(raft, conn);
1736 }
1737
1738 /* Close unneeded sessions. */
1739 struct raft_conn *next;
1740 LIST_FOR_EACH_SAFE (conn, next, list_node, &raft->conns) {
1741 if (!raft_conn_should_stay_open(raft, conn)) {
1742 raft_conn_close(conn);
1743 }
1744 }
1745
1746 /* Open needed sessions. */
1747 struct raft_server *server;
1748 HMAP_FOR_EACH (server, hmap_node, &raft->servers) {
1749 raft_open_conn(raft, server->address, &server->sid);
1750 }
1751 if (raft->joining) {
1752 const char *address;
1753 SSET_FOR_EACH (address, &raft->remote_addresses) {
1754 raft_open_conn(raft, address, NULL);
1755 }
1756 }
1757
1758 if (!raft->joining && time_msec() >= raft->election_timeout) {
1759 raft_start_election(raft, false);
1760 }
1761
1762 if (raft->leaving && time_msec() >= raft->leave_timeout) {
1763 raft_send_remove_server_requests(raft);
1764 }
1765
1766 if (raft->joining && time_msec() >= raft->join_timeout) {
1767 raft->join_timeout = time_msec() + 1000;
1768 LIST_FOR_EACH (conn, list_node, &raft->conns) {
1769 raft_send_add_server_request(raft, conn);
1770 }
1771 }
1772
1773 if (time_msec() >= raft->ping_timeout) {
1774 if (raft->role == RAFT_LEADER) {
1775 raft_send_heartbeats(raft);
1776 } else {
1777 long long int now = time_msec();
1778 struct raft_command *cmd, *next_cmd;
1779 HMAP_FOR_EACH_SAFE (cmd, next_cmd, hmap_node, &raft->commands) {
1780 if (cmd->timestamp
1781 && now - cmd->timestamp > ELECTION_BASE_MSEC) {
1782 raft_command_complete(raft, cmd, RAFT_CMD_TIMEOUT);
1783 }
1784 }
1785 }
1786 raft->ping_timeout = time_msec() + PING_TIME_MSEC;
1787 }
1788
1789 /* Do this only at the end; if we did it as soon as we set raft->left or
1790 * raft->failed in handling the RemoveServerReply, then it could easily
1791 * cause references to freed memory in RPC sessions, etc. */
1792 if (raft->left || raft->failed) {
1793 raft_close__(raft);
1794 }
1795}
1796
1797static void
1798raft_wait_session(struct jsonrpc_session *js)
1799{
1800 if (js) {
1801 jsonrpc_session_wait(js);
1802 jsonrpc_session_recv_wait(js);
1803 }
1804}
1805
1806/* Causes the next call to poll_block() to wake up when 'raft' needs to do
1807 * something. */
1808void
1809raft_wait(struct raft *raft)
1810{
1811 if (raft->left || raft->failed) {
1812 return;
1813 }
1814
1815 raft_waiters_wait(raft);
1816
1817 if (raft->listener) {
1818 pstream_wait(raft->listener);
1819 } else {
1820 poll_timer_wait_until(raft->listen_backoff);
1821 }
1822
1823 struct raft_conn *conn;
1824 LIST_FOR_EACH (conn, list_node, &raft->conns) {
1825 raft_wait_session(conn->js);
1826 }
1827
1828 if (!raft->joining) {
1829 poll_timer_wait_until(raft->election_timeout);
1830 } else {
1831 poll_timer_wait_until(raft->join_timeout);
1832 }
1833 if (raft->leaving) {
1834 poll_timer_wait_until(raft->leave_timeout);
1835 }
1836 if (raft->role == RAFT_LEADER || !hmap_is_empty(&raft->commands)) {
1837 poll_timer_wait_until(raft->ping_timeout);
1838 }
1839}
1840
1841static struct raft_waiter *
1842raft_waiter_create(struct raft *raft, enum raft_waiter_type type,
1843 bool start_commit)
1844{
1845 struct raft_waiter *w = xzalloc(sizeof *w);
1846 ovs_list_push_back(&raft->waiters, &w->list_node);
1847 w->commit_ticket = start_commit ? ovsdb_log_commit_start(raft->log) : 0;
1848 w->type = type;
1849 return w;
1850}
1851
1852/* Returns a human-readable representation of 'status' (or NULL if 'status' is
1853 * invalid). */
1854const char *
1855raft_command_status_to_string(enum raft_command_status status)
1856{
1857 switch (status) {
1858 case RAFT_CMD_INCOMPLETE:
1859 return "operation still in progress";
1860 case RAFT_CMD_SUCCESS:
1861 return "success";
1862 case RAFT_CMD_NOT_LEADER:
1863 return "not leader";
1864 case RAFT_CMD_BAD_PREREQ:
1865 return "prerequisite check failed";
1866 case RAFT_CMD_LOST_LEADERSHIP:
1867 return "lost leadership";
1868 case RAFT_CMD_SHUTDOWN:
1869 return "server shutdown";
1870 case RAFT_CMD_IO_ERROR:
1871 return "I/O error";
1872 case RAFT_CMD_TIMEOUT:
1873 return "timeout";
1874 default:
1875 return NULL;
1876 }
1877}
1878
1879/* Converts human-readable status in 's' into status code in '*statusp'.
1880 * Returns true if successful, false if 's' is unknown. */
1881bool
1882raft_command_status_from_string(const char *s,
1883 enum raft_command_status *statusp)
1884{
1885 for (enum raft_command_status status = 0; ; status++) {
1886 const char *s2 = raft_command_status_to_string(status);
1887 if (!s2) {
1888 *statusp = 0;
1889 return false;
1890 } else if (!strcmp(s, s2)) {
1891 *statusp = status;
1892 return true;
1893 }
1894 }
1895}
1896
1897static const struct uuid *
1898raft_get_eid(const struct raft *raft, uint64_t index)
1899{
1900 for (; index >= raft->log_start; index--) {
1901 const struct raft_entry *e = raft_get_entry(raft, index);
1902 if (e->data) {
1903 return &e->eid;
1904 }
1905 }
1906 return &raft->snap.eid;
1907}
1908
1909static const struct uuid *
1910raft_current_eid(const struct raft *raft)
1911{
1912 return raft_get_eid(raft, raft->log_end - 1);
1913}
1914
1915static struct raft_command *
1916raft_command_create_completed(enum raft_command_status status)
1917{
1918 ovs_assert(status != RAFT_CMD_INCOMPLETE);
1919
1920 struct raft_command *cmd = xzalloc(sizeof *cmd);
1921 cmd->n_refs = 1;
1922 cmd->status = status;
1923 return cmd;
1924}
1925
1926static struct raft_command *
1927raft_command_create_incomplete(struct raft *raft, uint64_t index)
1928{
1929 struct raft_command *cmd = xzalloc(sizeof *cmd);
1930 cmd->n_refs = 2; /* One for client, one for raft->commands. */
1931 cmd->index = index;
1932 cmd->status = RAFT_CMD_INCOMPLETE;
1933 hmap_insert(&raft->commands, &cmd->hmap_node, cmd->index);
1934 return cmd;
1935}
1936
1937static struct raft_command * OVS_WARN_UNUSED_RESULT
1938raft_command_initiate(struct raft *raft,
1939 const struct json *data, const struct json *servers,
1940 const struct uuid *eid)
1941{
1942 /* Write to local log. */
1943 uint64_t index = raft->log_end;
1944 if (!raft_handle_write_error(
1945 raft, raft_write_entry(
1946 raft, raft->term, json_nullable_clone(data), eid,
1947 json_nullable_clone(servers)))) {
1948 return raft_command_create_completed(RAFT_CMD_IO_ERROR);
1949 }
1950
1951 struct raft_command *cmd = raft_command_create_incomplete(raft, index);
1952 if (eid) {
1953 cmd->eid = *eid;
1954 }
1955
1956 raft_waiter_create(raft, RAFT_W_ENTRY, true)->entry.index = cmd->index;
1957
1958 /* Write to remote logs. */
1959 struct raft_server *s;
1960 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
1961 if (!uuid_equals(&s->sid, &raft->sid) && s->next_index == index) {
1962 raft_send_append_request(raft, s, 1, "execute command");
1963 s->next_index++;
1964 }
1965 }
1966
1967 return cmd;
1968}
1969
1970static struct raft_command * OVS_WARN_UNUSED_RESULT
1971raft_command_execute__(struct raft *raft,
1972 const struct json *data, const struct json *servers,
1973 const struct uuid *prereq, struct uuid *result)
1974{
1975 if (raft->joining || raft->leaving || raft->left || raft->failed) {
1976 return raft_command_create_completed(RAFT_CMD_SHUTDOWN);
1977 }
1978
1979 if (raft->role != RAFT_LEADER) {
1980 /* Consider proxying the command to the leader. We can only do that if
1981 * we know the leader and the command does not change the set of
1982 * servers. We do not proxy commands without prerequisites, even
1983 * though we could, because in an OVSDB context a log entry doesn't
1984 * make sense without context. */
1985 if (servers || !data
1986 || raft->role != RAFT_FOLLOWER || uuid_is_zero(&raft->leader_sid)
1987 || !prereq) {
1988 return raft_command_create_completed(RAFT_CMD_NOT_LEADER);
1989 }
1990 }
1991
1992 struct uuid eid = data ? uuid_random() : UUID_ZERO;
1993 if (result) {
1994 *result = eid;
1995 }
1996
1997 if (raft->role != RAFT_LEADER) {
1998 const union raft_rpc rpc = {
1999 .execute_command_request = {
2000 .common = {
2001 .type = RAFT_RPC_EXECUTE_COMMAND_REQUEST,
2002 .sid = raft->leader_sid,
2003 },
2004 .data = CONST_CAST(struct json *, data),
2005 .prereq = *prereq,
2006 .result = eid,
2007 }
2008 };
2009 if (!raft_send(raft, &rpc)) {
2010 /* Couldn't send command, so it definitely failed. */
2011 return raft_command_create_completed(RAFT_CMD_NOT_LEADER);
2012 }
2013
2014 struct raft_command *cmd = raft_command_create_incomplete(raft, 0);
2015 cmd->timestamp = time_msec();
2016 cmd->eid = eid;
2017 return cmd;
2018 }
2019
2020 const struct uuid *current_eid = raft_current_eid(raft);
2021 if (prereq && !uuid_equals(prereq, current_eid)) {
2022 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
2023 VLOG_INFO_RL(&rl, "current entry eid "UUID_FMT" does not match "
2024 "prerequisite "UUID_FMT,
2025 UUID_ARGS(current_eid), UUID_ARGS(prereq));
2026 return raft_command_create_completed(RAFT_CMD_BAD_PREREQ);
2027 }
2028
2029 return raft_command_initiate(raft, data, servers, &eid);
2030}
2031
2032/* Initiates appending a log entry to 'raft'. The log entry consists of 'data'
2033 * and, if 'prereq' is nonnull, it is only added to the log if the previous
2034 * entry in the log has entry ID 'prereq'. If 'result' is nonnull, it is
2035 * populated with the entry ID for the new log entry.
2036 *
2037 * Returns a "struct raft_command" that may be used to track progress adding
2038 * the log entry. The caller must eventually free the returned structure, with
2039 * raft_command_unref(). */
2040struct raft_command * OVS_WARN_UNUSED_RESULT
2041raft_command_execute(struct raft *raft, const struct json *data,
2042 const struct uuid *prereq, struct uuid *result)
2043{
2044 return raft_command_execute__(raft, data, NULL, prereq, result);
2045}
2046
2047/* Returns the status of 'cmd'. */
2048enum raft_command_status
2049raft_command_get_status(const struct raft_command *cmd)
2050{
2051 ovs_assert(cmd->n_refs > 0);
2052 return cmd->status;
2053}
2054
2055/* Returns the index of the log entry at which 'cmd' was committed.
2056 *
2057 * This function works only with successful commands. */
2058uint64_t
2059raft_command_get_commit_index(const struct raft_command *cmd)
2060{
2061 ovs_assert(cmd->n_refs > 0);
2062 ovs_assert(cmd->status == RAFT_CMD_SUCCESS);
2063 return cmd->index;
2064}
2065
2066/* Frees 'cmd'. */
2067void
2068raft_command_unref(struct raft_command *cmd)
2069{
2070 if (cmd) {
2071 ovs_assert(cmd->n_refs > 0);
2072 if (!--cmd->n_refs) {
2073 free(cmd);
2074 }
2075 }
2076}
2077
2078/* Causes poll_block() to wake up when 'cmd' has status to report. */
2079void
2080raft_command_wait(const struct raft_command *cmd)
2081{
2082 if (cmd->status != RAFT_CMD_INCOMPLETE) {
2083 poll_immediate_wake();
2084 }
2085}
2086
2087static void
2088raft_command_complete(struct raft *raft,
2089 struct raft_command *cmd,
2090 enum raft_command_status status)
2091{
2092 if (!uuid_is_zero(&cmd->sid)) {
2093 uint64_t commit_index = status == RAFT_CMD_SUCCESS ? cmd->index : 0;
2094 raft_send_execute_command_reply(raft, &cmd->sid, &cmd->eid, status,
2095 commit_index);
2096 }
2097
2098 ovs_assert(cmd->status == RAFT_CMD_INCOMPLETE);
2099 ovs_assert(cmd->n_refs > 0);
2100 hmap_remove(&raft->commands, &cmd->hmap_node);
2101 cmd->status = status;
2102 raft_command_unref(cmd);
2103}
2104
2105static void
2106raft_complete_all_commands(struct raft *raft, enum raft_command_status status)
2107{
2108 struct raft_command *cmd, *next;
2109 HMAP_FOR_EACH_SAFE (cmd, next, hmap_node, &raft->commands) {
2110 raft_command_complete(raft, cmd, status);
2111 }
2112}
2113
2114static struct raft_command *
2115raft_find_command_by_index(struct raft *raft, uint64_t index)
2116{
2117 struct raft_command *cmd;
2118
2119 HMAP_FOR_EACH_IN_BUCKET (cmd, hmap_node, index, &raft->commands) {
2120 if (cmd->index == index) {
2121 return cmd;
2122 }
2123 }
2124 return NULL;
2125}
2126
2127static struct raft_command *
2128raft_find_command_by_eid(struct raft *raft, const struct uuid *eid)
2129{
2130 struct raft_command *cmd;
2131
2132 HMAP_FOR_EACH (cmd, hmap_node, &raft->commands) {
2133 if (uuid_equals(&cmd->eid, eid)) {
2134 return cmd;
2135 }
2136 }
2137 return NULL;
2138}
2139\f
2140#define RAFT_RPC(ENUM, NAME) \
2141 static void raft_handle_##NAME(struct raft *, const struct raft_##NAME *);
2142RAFT_RPC_TYPES
2143#undef RAFT_RPC
2144
2145static void
2146raft_handle_hello_request(struct raft *raft OVS_UNUSED,
2147 const struct raft_hello_request *hello OVS_UNUSED)
2148{
2149}
2150
2151/* 'sid' is the server being added. */
2152static void
2153raft_send_add_server_reply__(struct raft *raft, const struct uuid *sid,
2154 const char *address,
2155 bool success, const char *comment)
2156{
2157 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(10, 10);
2158 if (!VLOG_DROP_INFO(&rl)) {
2159 struct ds s = DS_EMPTY_INITIALIZER;
2160 char buf[SID_LEN + 1];
2161 ds_put_format(&s, "adding %s ("SID_FMT" at %s) "
2162 "to cluster "CID_FMT" %s",
2163 raft_get_nickname(raft, sid, buf, sizeof buf),
2164 SID_ARGS(sid), address, CID_ARGS(&raft->cid),
2165 success ? "succeeded" : "failed");
2166 if (comment) {
2167 ds_put_format(&s, " (%s)", comment);
2168 }
2169 VLOG_INFO("%s", ds_cstr(&s));
2170 ds_destroy(&s);
2171 }
2172
2173 union raft_rpc rpy = {
2174 .add_server_reply = {
2175 .common = {
2176 .type = RAFT_RPC_ADD_SERVER_REPLY,
2177 .sid = *sid,
2178 .comment = CONST_CAST(char *, comment),
2179 },
2180 .success = success,
2181 }
2182 };
2183
2184 struct sset *remote_addresses = &rpy.add_server_reply.remote_addresses;
2185 sset_init(remote_addresses);
2186 if (!raft->joining) {
2187 struct raft_server *s;
2188 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
2189 if (!uuid_equals(&s->sid, &raft->sid)) {
2190 sset_add(remote_addresses, s->address);
2191 }
2192 }
2193 }
2194
2195 raft_send(raft, &rpy);
2196
2197 sset_destroy(remote_addresses);
2198}
2199
2200static void
17bd4149
BP
2201raft_send_remove_server_reply_rpc(struct raft *raft,
2202 const struct uuid *dst_sid,
2203 const struct uuid *target_sid,
1b1d2e6d
BP
2204 bool success, const char *comment)
2205{
17bd4149
BP
2206 if (uuid_equals(&raft->sid, dst_sid)) {
2207 if (success && uuid_equals(&raft->sid, target_sid)) {
2208 raft_finished_leaving_cluster(raft);
2209 }
2210 return;
2211 }
2212
1b1d2e6d
BP
2213 const union raft_rpc rpy = {
2214 .remove_server_reply = {
2215 .common = {
2216 .type = RAFT_RPC_REMOVE_SERVER_REPLY,
17bd4149 2217 .sid = *dst_sid,
1b1d2e6d
BP
2218 .comment = CONST_CAST(char *, comment),
2219 },
17bd4149
BP
2220 .target_sid = (uuid_equals(dst_sid, target_sid)
2221 ? UUID_ZERO
2222 : *target_sid),
1b1d2e6d
BP
2223 .success = success,
2224 }
2225 };
2226 raft_send(raft, &rpy);
2227}
2228
2229static void
2230raft_send_remove_server_reply__(struct raft *raft,
2231 const struct uuid *target_sid,
2232 const struct uuid *requester_sid,
2233 struct unixctl_conn *requester_conn,
2234 bool success, const char *comment)
2235{
2236 struct ds s = DS_EMPTY_INITIALIZER;
2237 ds_put_format(&s, "request ");
2238 if (!uuid_is_zero(requester_sid)) {
2239 char buf[SID_LEN + 1];
2240 ds_put_format(&s, "by %s",
2241 raft_get_nickname(raft, requester_sid, buf, sizeof buf));
2242 } else {
2243 ds_put_cstr(&s, "via unixctl");
2244 }
2245 ds_put_cstr(&s, " to remove ");
2246 if (!requester_conn && uuid_equals(target_sid, requester_sid)) {
2247 ds_put_cstr(&s, "itself");
2248 } else {
2249 char buf[SID_LEN + 1];
2250 ds_put_cstr(&s, raft_get_nickname(raft, target_sid, buf, sizeof buf));
17bd4149
BP
2251 if (uuid_equals(target_sid, &raft->sid)) {
2252 ds_put_cstr(&s, " (ourselves)");
2253 }
1b1d2e6d
BP
2254 }
2255 ds_put_format(&s, " from cluster "CID_FMT" %s",
2256 CID_ARGS(&raft->cid),
2257 success ? "succeeded" : "failed");
2258 if (comment) {
2259 ds_put_format(&s, " (%s)", comment);
2260 }
2261
2262 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(10, 10);
2263 VLOG_INFO_RL(&rl, "%s", ds_cstr(&s));
2264
2265 /* Send RemoveServerReply to the requester (which could be a server or a
2266 * unixctl connection. Also always send it to the removed server; this
2267 * allows it to be sure that it's really removed and update its log and
2268 * disconnect permanently. */
2269 if (!uuid_is_zero(requester_sid)) {
17bd4149 2270 raft_send_remove_server_reply_rpc(raft, requester_sid, target_sid,
1b1d2e6d
BP
2271 success, comment);
2272 }
2273 if (!uuid_equals(requester_sid, target_sid)) {
17bd4149
BP
2274 raft_send_remove_server_reply_rpc(raft, target_sid, target_sid,
2275 success, comment);
1b1d2e6d
BP
2276 }
2277 if (requester_conn) {
2278 if (success) {
2279 unixctl_command_reply(requester_conn, ds_cstr(&s));
2280 } else {
2281 unixctl_command_reply_error(requester_conn, ds_cstr(&s));
2282 }
2283 }
2284
2285 ds_destroy(&s);
2286}
2287
2288static void
2289raft_send_add_server_reply(struct raft *raft,
2290 const struct raft_add_server_request *rq,
2291 bool success, const char *comment)
2292{
2293 return raft_send_add_server_reply__(raft, &rq->common.sid, rq->address,
2294 success, comment);
2295}
2296
2297static void
2298raft_send_remove_server_reply(struct raft *raft,
2299 const struct raft_remove_server_request *rq,
2300 bool success, const char *comment)
2301{
2302 return raft_send_remove_server_reply__(raft, &rq->sid, &rq->common.sid,
2303 rq->requester_conn, success,
2304 comment);
2305}
2306
2307static void
2308raft_become_follower(struct raft *raft)
2309{
2310 raft->leader_sid = UUID_ZERO;
2311 if (raft->role == RAFT_FOLLOWER) {
2312 return;
2313 }
2314
2315 raft->role = RAFT_FOLLOWER;
2316 raft_reset_timer(raft);
2317
2318 /* Notify clients about lost leadership.
2319 *
2320 * We do not reverse our changes to 'raft->servers' because the new
2321 * configuration is already part of the log. Possibly the configuration
2322 * log entry will not be committed, but until we know that we must use the
2323 * new configuration. Our AppendEntries processing will properly update
2324 * the server configuration later, if necessary. */
2325 struct raft_server *s;
2326 HMAP_FOR_EACH (s, hmap_node, &raft->add_servers) {
2327 raft_send_add_server_reply__(raft, &s->sid, s->address, false,
2328 RAFT_SERVER_LOST_LEADERSHIP);
2329 }
2330 if (raft->remove_server) {
2331 raft_send_remove_server_reply__(raft, &raft->remove_server->sid,
2332 &raft->remove_server->requester_sid,
2333 raft->remove_server->requester_conn,
2334 false, RAFT_SERVER_LOST_LEADERSHIP);
2335 raft_server_destroy(raft->remove_server);
2336 raft->remove_server = NULL;
2337 }
2338
2339 raft_complete_all_commands(raft, RAFT_CMD_LOST_LEADERSHIP);
2340}
2341
2342static void
2343raft_send_append_request(struct raft *raft,
2344 struct raft_server *peer, unsigned int n,
2345 const char *comment)
2346{
2347 ovs_assert(raft->role == RAFT_LEADER);
2348
2349 const union raft_rpc rq = {
2350 .append_request = {
2351 .common = {
2352 .type = RAFT_RPC_APPEND_REQUEST,
2353 .sid = peer->sid,
2354 .comment = CONST_CAST(char *, comment),
2355 },
2356 .term = raft->term,
2357 .prev_log_index = peer->next_index - 1,
2358 .prev_log_term = (peer->next_index - 1 >= raft->log_start
2359 ? raft->entries[peer->next_index - 1
2360 - raft->log_start].term
2361 : raft->snap.term),
2362 .leader_commit = raft->commit_index,
2363 .entries = &raft->entries[peer->next_index - raft->log_start],
2364 .n_entries = n,
2365 },
2366 };
2367 raft_send(raft, &rq);
2368}
2369
2370static void
2371raft_send_heartbeats(struct raft *raft)
2372{
2373 struct raft_server *s;
2374 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
2375 if (!uuid_equals(&raft->sid, &s->sid)) {
2376 raft_send_append_request(raft, s, 0, "heartbeat");
2377 }
2378 }
2379
2380 /* Send anyone waiting for a command to complete a ping to let them
2381 * know we're still working on it. */
2382 struct raft_command *cmd;
2383 HMAP_FOR_EACH (cmd, hmap_node, &raft->commands) {
2384 if (!uuid_is_zero(&cmd->sid)) {
2385 raft_send_execute_command_reply(raft, &cmd->sid,
2386 &cmd->eid,
2387 RAFT_CMD_INCOMPLETE, 0);
2388 }
2389 }
2390}
2391
2392/* Initializes the fields in 's' that represent the leader's view of the
2393 * server. */
2394static void
2395raft_server_init_leader(struct raft *raft, struct raft_server *s)
2396{
2397 s->next_index = raft->log_end;
2398 s->match_index = 0;
2399 s->phase = RAFT_PHASE_STABLE;
2400}
2401
2402static void
2403raft_become_leader(struct raft *raft)
2404{
2405 raft_complete_all_commands(raft, RAFT_CMD_LOST_LEADERSHIP);
2406
2407 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
2408 VLOG_INFO_RL(&rl, "term %"PRIu64": elected leader by %d+ of "
2409 "%"PRIuSIZE" servers", raft->term,
2410 raft->n_votes, hmap_count(&raft->servers));
2411
2412 ovs_assert(raft->role != RAFT_LEADER);
2413 raft->role = RAFT_LEADER;
2414 raft->leader_sid = raft->sid;
2415 raft->election_timeout = LLONG_MAX;
2416 raft->ping_timeout = time_msec() + PING_TIME_MSEC;
2417
2418 struct raft_server *s;
2419 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
2420 raft_server_init_leader(raft, s);
2421 }
2422
2423 raft_update_our_match_index(raft, raft->log_end - 1);
2424 raft_send_heartbeats(raft);
2425
2426 /* Write the fact that we are leader to the log. This is not used by the
2427 * algorithm (although it could be, for quick restart), but it is used for
2428 * offline analysis to check for conformance with the properties that Raft
2429 * guarantees. */
2430 struct raft_record r = {
2431 .type = RAFT_REC_LEADER,
2432 .term = raft->term,
2433 .sid = raft->sid,
2434 };
2435 ignore(ovsdb_log_write_and_free(raft->log, raft_record_to_json(&r)));
2436
2437 /* Initiate a no-op commit. Otherwise we might never find out what's in
2438 * the log. See section 6.4 item 1:
2439 *
2440 * The Leader Completeness Property guarantees that a leader has all
2441 * committed entries, but at the start of its term, it may not know
2442 * which those are. To find out, it needs to commit an entry from its
2443 * term. Raft handles this by having each leader commit a blank no-op
2444 * entry into the log at the start of its term. As soon as this no-op
2445 * entry is committed, the leader’s commit index will be at least as
2446 * large as any other servers’ during its term.
2447 */
2448 raft_command_unref(raft_command_execute__(raft, NULL, NULL, NULL, NULL));
2449}
2450
2451/* Processes term 'term' received as part of RPC 'common'. Returns true if the
2452 * caller should continue processing the RPC, false if the caller should reject
2453 * it due to a stale term. */
2454static bool
2455raft_receive_term__(struct raft *raft, const struct raft_rpc_common *common,
2456 uint64_t term)
2457{
2458 /* Section 3.3 says:
2459 *
2460 * Current terms are exchanged whenever servers communicate; if one
2461 * server’s current term is smaller than the other’s, then it updates
2462 * its current term to the larger value. If a candidate or leader
2463 * discovers that its term is out of date, it immediately reverts to
2464 * follower state. If a server receives a request with a stale term
2465 * number, it rejects the request.
2466 */
2467 if (term > raft->term) {
2468 if (!raft_set_term(raft, term, NULL)) {
2469 /* Failed to update the term to 'term'. */
2470 return false;
2471 }
2472 raft_become_follower(raft);
2473 } else if (term < raft->term) {
2474 char buf[SID_LEN + 1];
2475 VLOG_INFO("rejecting term %"PRIu64" < current term %"PRIu64" received "
2476 "in %s message from server %s",
2477 term, raft->term,
2478 raft_rpc_type_to_string(common->type),
2479 raft_get_nickname(raft, &common->sid, buf, sizeof buf));
2480 return false;
2481 }
2482 return true;
2483}
2484
2485static void
2486raft_get_servers_from_log(struct raft *raft, enum vlog_level level)
2487{
2488 const struct json *servers_json = raft->snap.servers;
2489 for (uint64_t index = raft->log_end - 1; index >= raft->log_start;
2490 index--) {
2491 struct raft_entry *e = &raft->entries[index - raft->log_start];
2492 if (e->servers) {
2493 servers_json = e->servers;
2494 break;
2495 }
2496 }
2497
2498 struct hmap servers;
2499 struct ovsdb_error *error = raft_servers_from_json(servers_json, &servers);
2500 ovs_assert(!error);
2501 raft_set_servers(raft, &servers, level);
2502 raft_servers_destroy(&servers);
2503}
2504
2505/* Truncates the log, so that raft->log_end becomes 'new_end'.
2506 *
2507 * Doesn't write anything to disk. In theory, we could truncate the on-disk
2508 * log file, but we don't have the right information to know how long it should
2509 * be. What we actually do is to append entries for older indexes to the
2510 * on-disk log; when we re-read it later, these entries truncate the log.
2511 *
2512 * Returns true if any of the removed log entries were server configuration
2513 * entries, false otherwise. */
2514static bool
2515raft_truncate(struct raft *raft, uint64_t new_end)
2516{
2517 ovs_assert(new_end >= raft->log_start);
2518 if (raft->log_end > new_end) {
2519 char buf[SID_LEN + 1];
2520 VLOG_INFO("%s truncating %"PRIu64 " entries from end of log",
2521 raft_get_nickname(raft, &raft->sid, buf, sizeof buf),
2522 raft->log_end - new_end);
2523 }
2524
2525 bool servers_changed = false;
2526 while (raft->log_end > new_end) {
2527 struct raft_entry *entry = &raft->entries[--raft->log_end
2528 - raft->log_start];
2529 if (entry->servers) {
2530 servers_changed = true;
2531 }
2532 raft_entry_uninit(entry);
2533 }
2534 return servers_changed;
2535}
2536
2537static const struct json *
2538raft_peek_next_entry(struct raft *raft, struct uuid *eid)
2539{
2540 /* Invariant: log_start - 2 <= last_applied <= commit_index < log_end. */
2541 ovs_assert(raft->log_start <= raft->last_applied + 2);
2542 ovs_assert(raft->last_applied <= raft->commit_index);
2543 ovs_assert(raft->commit_index < raft->log_end);
2544
2545 if (raft->joining || raft->failed) {
2546 return NULL;
2547 }
2548
2549 if (raft->log_start == raft->last_applied + 2) {
2550 *eid = raft->snap.eid;
2551 return raft->snap.data;
2552 }
2553
2554 while (raft->last_applied < raft->commit_index) {
2555 const struct raft_entry *e = raft_get_entry(raft,
2556 raft->last_applied + 1);
2557 if (e->data) {
2558 *eid = e->eid;
2559 return e->data;
2560 }
2561 raft->last_applied++;
2562 }
2563 return NULL;
2564}
2565
2566static const struct json *
2567raft_get_next_entry(struct raft *raft, struct uuid *eid)
2568{
2569 const struct json *data = raft_peek_next_entry(raft, eid);
2570 if (data) {
2571 raft->last_applied++;
2572 }
2573 return data;
2574}
2575
2576static void
2577raft_update_commit_index(struct raft *raft, uint64_t new_commit_index)
2578{
2579 if (new_commit_index <= raft->commit_index) {
2580 return;
2581 }
2582
2583 if (raft->role == RAFT_LEADER) {
2584 while (raft->commit_index < new_commit_index) {
2585 uint64_t index = ++raft->commit_index;
2586 const struct raft_entry *e = raft_get_entry(raft, index);
1b1d2e6d
BP
2587 if (e->data) {
2588 struct raft_command *cmd
2589 = raft_find_command_by_index(raft, index);
2590 if (cmd) {
2591 raft_command_complete(raft, cmd, RAFT_CMD_SUCCESS);
2592 }
2593 }
8b37ed75
BP
2594 if (e->servers) {
2595 /* raft_run_reconfigure() can write a new Raft entry, which can
2596 * reallocate raft->entries, which would invalidate 'e', so
2597 * this case must be last, after the one for 'e->data'. */
2598 raft_run_reconfigure(raft);
2599 }
1b1d2e6d
BP
2600 }
2601 } else {
2602 raft->commit_index = new_commit_index;
2603 }
2604
2605 /* Write the commit index to the log. The next time we restart, this
2606 * allows us to start exporting a reasonably fresh log, instead of a log
2607 * that only contains the snapshot. */
2608 struct raft_record r = {
2609 .type = RAFT_REC_COMMIT_INDEX,
2610 .commit_index = raft->commit_index,
2611 };
2612 ignore(ovsdb_log_write_and_free(raft->log, raft_record_to_json(&r)));
2613}
2614
2615/* This doesn't use rq->entries (but it does use rq->n_entries). */
2616static void
2617raft_send_append_reply(struct raft *raft, const struct raft_append_request *rq,
2618 enum raft_append_result result, const char *comment)
2619{
2620 /* Figure 3.1: "If leaderCommit > commitIndex, set commitIndex =
2621 * min(leaderCommit, index of last new entry)" */
2622 if (result == RAFT_APPEND_OK && rq->leader_commit > raft->commit_index) {
2623 raft_update_commit_index(
2624 raft, MIN(rq->leader_commit, rq->prev_log_index + rq->n_entries));
2625 }
2626
2627 /* Send reply. */
2628 union raft_rpc reply = {
2629 .append_reply = {
2630 .common = {
2631 .type = RAFT_RPC_APPEND_REPLY,
2632 .sid = rq->common.sid,
2633 .comment = CONST_CAST(char *, comment),
2634 },
2635 .term = raft->term,
2636 .log_end = raft->log_end,
2637 .prev_log_index = rq->prev_log_index,
2638 .prev_log_term = rq->prev_log_term,
2639 .n_entries = rq->n_entries,
2640 .result = result,
2641 }
2642 };
2643 raft_send(raft, &reply);
2644}
2645
2646/* If 'prev_log_index' exists in 'raft''s log, in term 'prev_log_term', returns
2647 * NULL. Otherwise, returns an explanation for the mismatch. */
2648static const char *
2649match_index_and_term(const struct raft *raft,
2650 uint64_t prev_log_index, uint64_t prev_log_term)
2651{
2652 if (prev_log_index < raft->log_start - 1) {
2653 return "mismatch before start of log";
2654 } else if (prev_log_index == raft->log_start - 1) {
2655 if (prev_log_term != raft->snap.term) {
2656 return "prev_term mismatch";
2657 }
2658 } else if (prev_log_index < raft->log_end) {
2659 if (raft->entries[prev_log_index - raft->log_start].term
2660 != prev_log_term) {
2661 return "term mismatch";
2662 }
2663 } else {
2664 /* prev_log_index >= raft->log_end */
2665 return "mismatch past end of log";
2666 }
2667 return NULL;
2668}
2669
2670static void
2671raft_handle_append_entries(struct raft *raft,
2672 const struct raft_append_request *rq,
2673 uint64_t prev_log_index, uint64_t prev_log_term,
2674 const struct raft_entry *entries,
2675 unsigned int n_entries)
2676{
2677 /* Section 3.5: "When sending an AppendEntries RPC, the leader includes
2678 * the index and term of the entry in its log that immediately precedes
2679 * the new entries. If the follower does not find an entry in its log
2680 * with the same index and term, then it refuses the new entries." */
2681 const char *mismatch = match_index_and_term(raft, prev_log_index,
2682 prev_log_term);
2683 if (mismatch) {
2684 VLOG_INFO("rejecting append_request because previous entry "
2685 "%"PRIu64",%"PRIu64" not in local log (%s)",
2686 prev_log_term, prev_log_index, mismatch);
2687 raft_send_append_reply(raft, rq, RAFT_APPEND_INCONSISTENCY, mismatch);
2688 return;
2689 }
2690
2691 /* Figure 3.1: "If an existing entry conflicts with a new one (same
2692 * index but different terms), delete the existing entry and all that
2693 * follow it." */
2694 unsigned int i;
2695 bool servers_changed = false;
2696 for (i = 0; ; i++) {
2697 if (i >= n_entries) {
2698 /* No change. */
2699 if (rq->common.comment
2700 && !strcmp(rq->common.comment, "heartbeat")) {
2701 raft_send_append_reply(raft, rq, RAFT_APPEND_OK, "heartbeat");
2702 } else {
2703 raft_send_append_reply(raft, rq, RAFT_APPEND_OK, "no change");
2704 }
2705 return;
2706 }
2707
2708 uint64_t log_index = (prev_log_index + 1) + i;
2709 if (log_index >= raft->log_end) {
2710 break;
2711 }
2712 if (raft->entries[log_index - raft->log_start].term
2713 != entries[i].term) {
2714 if (raft_truncate(raft, log_index)) {
2715 servers_changed = true;
2716 }
2717 break;
2718 }
2719 }
2720
2721 /* Figure 3.1: "Append any entries not already in the log." */
2722 struct ovsdb_error *error = NULL;
2723 bool any_written = false;
2724 for (; i < n_entries; i++) {
2725 const struct raft_entry *e = &entries[i];
2726 error = raft_write_entry(raft, e->term,
2727 json_nullable_clone(e->data), &e->eid,
2728 json_nullable_clone(e->servers));
2729 if (error) {
2730 break;
2731 }
2732 any_written = true;
2733 if (e->servers) {
2734 servers_changed = true;
2735 }
2736 }
2737
2738 if (any_written) {
2739 raft_waiter_create(raft, RAFT_W_ENTRY, true)->entry.index
2740 = raft->log_end - 1;
2741 }
2742 if (servers_changed) {
2743 /* The set of servers might have changed; check. */
2744 raft_get_servers_from_log(raft, VLL_INFO);
2745 }
2746
2747 if (error) {
2748 char *s = ovsdb_error_to_string_free(error);
2749 VLOG_ERR("%s", s);
2750 free(s);
2751 raft_send_append_reply(raft, rq, RAFT_APPEND_IO_ERROR, "I/O error");
2752 return;
2753 }
2754
2755 raft_send_append_reply(raft, rq, RAFT_APPEND_OK, "log updated");
2756}
2757
2758static bool
2759raft_update_leader(struct raft *raft, const struct uuid *sid)
2760{
2761 if (raft->role == RAFT_LEADER) {
2762 char buf[SID_LEN + 1];
2763 VLOG_ERR("this server is leader but server %s claims to be",
2764 raft_get_nickname(raft, sid, buf, sizeof buf));
2765 return false;
2766 } else if (!uuid_equals(sid, &raft->leader_sid)) {
2767 if (!uuid_is_zero(&raft->leader_sid)) {
2768 char buf1[SID_LEN + 1];
2769 char buf2[SID_LEN + 1];
2770 VLOG_ERR("leader for term %"PRIu64" changed from %s to %s",
2771 raft->term,
2772 raft_get_nickname(raft, &raft->leader_sid,
2773 buf1, sizeof buf1),
2774 raft_get_nickname(raft, sid, buf2, sizeof buf2));
2775 } else {
2776 char buf[SID_LEN + 1];
2777 VLOG_INFO("server %s is leader for term %"PRIu64,
2778 raft_get_nickname(raft, sid, buf, sizeof buf),
2779 raft->term);
2780 }
2781 raft->leader_sid = *sid;
2782
2783 /* Record the leader to the log. This is not used by the algorithm
2784 * (although it could be, for quick restart), but it is used for
2785 * offline analysis to check for conformance with the properties
2786 * that Raft guarantees. */
2787 struct raft_record r = {
2788 .type = RAFT_REC_LEADER,
2789 .term = raft->term,
2790 .sid = *sid,
2791 };
2792 ignore(ovsdb_log_write_and_free(raft->log, raft_record_to_json(&r)));
2793 }
2794 return true;
2795}
2796
2797static void
2798raft_handle_append_request(struct raft *raft,
2799 const struct raft_append_request *rq)
2800{
2801 /* We do not check whether the server that sent the request is part of the
2802 * cluster. As section 4.1 says, "A server accepts AppendEntries requests
2803 * from a leader that is not part of the server’s latest configuration.
2804 * Otherwise, a new server could never be added to the cluster (it would
2805 * never accept any log entries preceding the configuration entry that adds
2806 * the server)." */
2807 if (!raft_update_leader(raft, &rq->common.sid)) {
2808 raft_send_append_reply(raft, rq, RAFT_APPEND_INCONSISTENCY,
2809 "usurped leadership");
2810 return;
2811 }
2812 raft_reset_timer(raft);
2813
2814 /* First check for the common case, where the AppendEntries request is
2815 * entirely for indexes covered by 'log_start' ... 'log_end - 1', something
2816 * like this:
2817 *
2818 * rq->prev_log_index
2819 * | first_entry_index
2820 * | | nth_entry_index
2821 * | | |
2822 * v v v
2823 * +---+---+---+---+
2824 * T | T | T | T | T |
2825 * +---+-------+---+
2826 * +---+---+---+---+
2827 * T | T | T | T | T |
2828 * +---+---+---+---+
2829 * ^ ^
2830 * | |
2831 * log_start log_end
2832 * */
2833 uint64_t first_entry_index = rq->prev_log_index + 1;
2834 uint64_t nth_entry_index = rq->prev_log_index + rq->n_entries;
2835 if (OVS_LIKELY(first_entry_index >= raft->log_start)) {
2836 raft_handle_append_entries(raft, rq,
2837 rq->prev_log_index, rq->prev_log_term,
2838 rq->entries, rq->n_entries);
2839 return;
2840 }
2841
2842 /* Now a series of checks for odd cases, where the AppendEntries request
2843 * extends earlier than the beginning of our log, into the log entries
2844 * discarded by the most recent snapshot. */
2845
2846 /*
2847 * Handle the case where the indexes covered by rq->entries[] are entirely
2848 * disjoint with 'log_start - 1' ... 'log_end - 1', as shown below. So,
2849 * everything in the AppendEntries request must already have been
2850 * committed, and we might as well return true.
2851 *
2852 * rq->prev_log_index
2853 * | first_entry_index
2854 * | | nth_entry_index
2855 * | | |
2856 * v v v
2857 * +---+---+---+---+
2858 * T | T | T | T | T |
2859 * +---+-------+---+
2860 * +---+---+---+---+
2861 * T | T | T | T | T |
2862 * +---+---+---+---+
2863 * ^ ^
2864 * | |
2865 * log_start log_end
2866 */
2867 if (nth_entry_index < raft->log_start - 1) {
2868 raft_send_append_reply(raft, rq, RAFT_APPEND_OK,
2869 "append before log start");
2870 return;
2871 }
2872
2873 /*
2874 * Handle the case where the last entry in rq->entries[] has the same index
2875 * as 'log_start - 1', so we can compare their terms:
2876 *
2877 * rq->prev_log_index
2878 * | first_entry_index
2879 * | | nth_entry_index
2880 * | | |
2881 * v v v
2882 * +---+---+---+---+
2883 * T | T | T | T | T |
2884 * +---+-------+---+
2885 * +---+---+---+---+
2886 * T | T | T | T | T |
2887 * +---+---+---+---+
2888 * ^ ^
2889 * | |
2890 * log_start log_end
2891 *
2892 * There's actually a sub-case where rq->n_entries == 0, in which we
2893 * compare rq->prev_term:
2894 *
2895 * rq->prev_log_index
2896 * |
2897 * |
2898 * |
2899 * v
2900 * T
2901 *
2902 * +---+---+---+---+
2903 * T | T | T | T | T |
2904 * +---+---+---+---+
2905 * ^ ^
2906 * | |
2907 * log_start log_end
2908 */
2909 if (nth_entry_index == raft->log_start - 1) {
2910 if (rq->n_entries
2911 ? raft->snap.term == rq->entries[rq->n_entries - 1].term
2912 : raft->snap.term == rq->prev_log_term) {
2913 raft_send_append_reply(raft, rq, RAFT_APPEND_OK, "no change");
2914 } else {
2915 raft_send_append_reply(raft, rq, RAFT_APPEND_INCONSISTENCY,
2916 "term mismatch");
2917 }
2918 return;
2919 }
2920
2921 /*
2922 * We now know that the data in rq->entries[] overlaps the data in
2923 * raft->entries[], as shown below, with some positive 'ofs':
2924 *
2925 * rq->prev_log_index
2926 * | first_entry_index
2927 * | | nth_entry_index
2928 * | | |
2929 * v v v
2930 * +---+---+---+---+---+
2931 * T | T | T | T | T | T |
2932 * +---+-------+---+---+
2933 * +---+---+---+---+
2934 * T | T | T | T | T |
2935 * +---+---+---+---+
2936 * ^ ^
2937 * | |
2938 * log_start log_end
2939 *
2940 * |<-- ofs -->|
2941 *
2942 * We transform this into the following by trimming the first 'ofs'
2943 * elements off of rq->entries[], ending up with the following. Notice how
2944 * we retain the term but not the data for rq->entries[ofs - 1]:
2945 *
2946 * first_entry_index + ofs - 1
2947 * | first_entry_index + ofs
2948 * | | nth_entry_index + ofs
2949 * | | |
2950 * v v v
2951 * +---+---+
2952 * T | T | T |
2953 * +---+---+
2954 * +---+---+---+---+
2955 * T | T | T | T | T |
2956 * +---+---+---+---+
2957 * ^ ^
2958 * | |
2959 * log_start log_end
2960 */
2961 uint64_t ofs = raft->log_start - first_entry_index;
2962 raft_handle_append_entries(raft, rq,
2963 raft->log_start - 1, rq->entries[ofs - 1].term,
2964 &rq->entries[ofs], rq->n_entries - ofs);
2965}
2966
2967/* Returns true if 'raft' has another log entry or snapshot to read. */
2968bool
2969raft_has_next_entry(const struct raft *raft_)
2970{
2971 struct raft *raft = CONST_CAST(struct raft *, raft_);
2972 struct uuid eid;
2973 return raft_peek_next_entry(raft, &eid) != NULL;
2974}
2975
2976/* Returns the next log entry or snapshot from 'raft', or NULL if there are
2977 * none left to read. Stores the entry ID of the log entry in '*eid'. Stores
2978 * true in '*is_snapshot' if the returned data is a snapshot, false if it is a
2979 * log entry. */
2980const struct json *
2981raft_next_entry(struct raft *raft, struct uuid *eid, bool *is_snapshot)
2982{
2983 const struct json *data = raft_get_next_entry(raft, eid);
2984 *is_snapshot = data == raft->snap.data;
2985 return data;
2986}
2987
2988/* Returns the log index of the last-read snapshot or log entry. */
2989uint64_t
2990raft_get_applied_index(const struct raft *raft)
2991{
2992 return raft->last_applied;
2993}
2994
2995/* Returns the log index of the last snapshot or log entry that is available to
2996 * be read. */
2997uint64_t
2998raft_get_commit_index(const struct raft *raft)
2999{
3000 return raft->commit_index;
3001}
3002
3003static struct raft_server *
3004raft_find_peer(struct raft *raft, const struct uuid *uuid)
3005{
3006 struct raft_server *s = raft_find_server(raft, uuid);
3007 return s && !uuid_equals(&raft->sid, &s->sid) ? s : NULL;
3008}
3009
3010static struct raft_server *
3011raft_find_new_server(struct raft *raft, const struct uuid *uuid)
3012{
3013 return raft_server_find(&raft->add_servers, uuid);
3014}
3015
3016/* Figure 3.1: "If there exists an N such that N > commitIndex, a
3017 * majority of matchIndex[i] >= N, and log[N].term == currentTerm, set
3018 * commitIndex = N (sections 3.5 and 3.6)." */
3019static void
3020raft_consider_updating_commit_index(struct raft *raft)
3021{
3022 /* This loop cannot just bail out when it comes across a log entry that
3023 * does not match the criteria. For example, Figure 3.7(d2) shows a
3024 * case where the log entry for term 2 cannot be committed directly
3025 * (because it is not for the current term) but it can be committed as
3026 * a side effect of commit the entry for term 4 (the current term).
3027 * XXX Is there a more efficient way to do this? */
3028 ovs_assert(raft->role == RAFT_LEADER);
3029
3030 uint64_t new_commit_index = raft->commit_index;
3031 for (uint64_t idx = MAX(raft->commit_index + 1, raft->log_start);
3032 idx < raft->log_end; idx++) {
3033 if (raft->entries[idx - raft->log_start].term == raft->term) {
3034 size_t count = 0;
3035 struct raft_server *s2;
3036 HMAP_FOR_EACH (s2, hmap_node, &raft->servers) {
3037 if (s2->match_index >= idx) {
3038 count++;
3039 }
3040 }
3041 if (count > hmap_count(&raft->servers) / 2) {
3042 VLOG_DBG("index %"PRIu64" committed to %"PRIuSIZE" servers, "
3043 "applying", idx, count);
3044 new_commit_index = idx;
3045 }
3046 }
3047 }
3048 raft_update_commit_index(raft, new_commit_index);
3049}
3050
3051static void
3052raft_update_match_index(struct raft *raft, struct raft_server *s,
3053 uint64_t min_index)
3054{
3055 ovs_assert(raft->role == RAFT_LEADER);
3056 if (min_index > s->match_index) {
3057 s->match_index = min_index;
3058 raft_consider_updating_commit_index(raft);
3059 }
3060}
3061
3062static void
3063raft_update_our_match_index(struct raft *raft, uint64_t min_index)
3064{
e8208c66
BP
3065 struct raft_server *server = raft_find_server(raft, &raft->sid);
3066 if (server) {
3067 raft_update_match_index(raft, server, min_index);
3068 }
1b1d2e6d
BP
3069}
3070
3071static void
3072raft_send_install_snapshot_request(struct raft *raft,
3073 const struct raft_server *s,
3074 const char *comment)
3075{
3076 union raft_rpc rpc = {
3077 .install_snapshot_request = {
3078 .common = {
3079 .type = RAFT_RPC_INSTALL_SNAPSHOT_REQUEST,
3080 .sid = s->sid,
3081 .comment = CONST_CAST(char *, comment),
3082 },
3083 .term = raft->term,
3084 .last_index = raft->log_start - 1,
3085 .last_term = raft->snap.term,
3086 .last_servers = raft->snap.servers,
3087 .last_eid = raft->snap.eid,
3088 .data = raft->snap.data,
3089 }
3090 };
3091 raft_send(raft, &rpc);
3092}
3093
3094static void
3095raft_handle_append_reply(struct raft *raft,
3096 const struct raft_append_reply *rpy)
3097{
3098 if (raft->role != RAFT_LEADER) {
3099 VLOG_INFO("rejected append_reply (not leader)");
3100 return;
3101 }
3102
3103 /* Most commonly we'd be getting an AppendEntries reply from a configured
3104 * server (e.g. a peer), but we can also get them from servers in the
3105 * process of being added. */
3106 struct raft_server *s = raft_find_peer(raft, &rpy->common.sid);
3107 if (!s) {
3108 s = raft_find_new_server(raft, &rpy->common.sid);
3109 if (!s) {
3110 VLOG_INFO("rejected append_reply from unknown server "SID_FMT,
3111 SID_ARGS(&rpy->common.sid));
3112 return;
3113 }
3114 }
3115
3116 if (rpy->result == RAFT_APPEND_OK) {
3117 /* Figure 3.1: "If successful, update nextIndex and matchIndex for
3118 * follower (section 3.5)." */
3119 uint64_t min_index = rpy->prev_log_index + rpy->n_entries + 1;
3120 if (s->next_index < min_index) {
3121 s->next_index = min_index;
3122 }
3123 raft_update_match_index(raft, s, min_index - 1);
3124 } else {
3125 /* Figure 3.1: "If AppendEntries fails because of log inconsistency,
3126 * decrement nextIndex and retry (section 3.5)."
3127 *
3128 * We also implement the optimization suggested in section 4.2.1:
3129 * "Various approaches can make nextIndex converge to its correct value
3130 * more quickly, including those described in Chapter 3. The simplest
3131 * approach to solving this particular problem of adding a new server,
3132 * however, is to have followers return the length of their logs in the
3133 * AppendEntries response; this allows the leader to cap the follower’s
3134 * nextIndex accordingly." */
3135 s->next_index = (s->next_index > 0
3136 ? MIN(s->next_index - 1, rpy->log_end)
3137 : 0);
3138
3139 if (rpy->result == RAFT_APPEND_IO_ERROR) {
3140 /* Append failed but not because of a log inconsistency. Because
3141 * of the I/O error, there's no point in re-sending the append
3142 * immediately. */
3143 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
3144 VLOG_INFO_RL(&rl, "%s reported I/O error", s->nickname);
3145 return;
3146 }
3147 }
3148
3149 /*
3150 * Our behavior here must depend on the value of next_index relative to
3151 * log_start and log_end. There are three cases:
3152 *
3153 * Case 1 | Case 2 | Case 3
3154 * <---------------->|<------------->|<------------------>
3155 * | |
3156 *
3157 * +---+---+---+---+
3158 * T | T | T | T | T |
3159 * +---+---+---+---+
3160 * ^ ^
3161 * | |
3162 * log_start log_end
3163 */
3164 if (s->next_index < raft->log_start) {
3165 /* Case 1. */
3166 raft_send_install_snapshot_request(raft, s, NULL);
3167 } else if (s->next_index < raft->log_end) {
3168 /* Case 2. */
3169 raft_send_append_request(raft, s, 1, NULL);
3170 } else {
3171 /* Case 3. */
3172 if (s->phase == RAFT_PHASE_CATCHUP) {
3173 s->phase = RAFT_PHASE_CAUGHT_UP;
3174 raft_run_reconfigure(raft);
3175 }
3176 }
3177}
3178
3179static bool
3180raft_should_suppress_disruptive_server(struct raft *raft,
3181 const union raft_rpc *rpc)
3182{
3183 if (rpc->type != RAFT_RPC_VOTE_REQUEST) {
3184 return false;
3185 }
3186
3187 /* Section 4.2.3 "Disruptive Servers" says:
3188 *
3189 * ...if a server receives a RequestVote request within the minimum
3190 * election timeout of hearing from a current leader, it does not update
3191 * its term or grant its vote...
3192 *
3193 * ...This change conflicts with the leadership transfer mechanism as
3194 * described in Chapter 3, in which a server legitimately starts an
3195 * election without waiting an election timeout. In that case,
3196 * RequestVote messages should be processed by other servers even when
3197 * they believe a current cluster leader exists. Those RequestVote
3198 * requests can include a special flag to indicate this behavior (“I
3199 * have permission to disrupt the leader--it told me to!”).
3200 *
3201 * This clearly describes how the followers should act, but not the leader.
3202 * We just ignore vote requests that arrive at a current leader. This
3203 * seems to be fairly safe, since a majority other than the current leader
3204 * can still elect a new leader and the first AppendEntries from that new
3205 * leader will depose the current leader. */
3206 const struct raft_vote_request *rq = raft_vote_request_cast(rpc);
3207 if (rq->leadership_transfer) {
3208 return false;
3209 }
3210
3211 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
3212 long long int now = time_msec();
3213 switch (raft->role) {
3214 case RAFT_LEADER:
3215 VLOG_WARN_RL(&rl, "ignoring vote request received as leader");
3216 return true;
3217
3218 case RAFT_FOLLOWER:
3219 if (now < raft->election_base + ELECTION_BASE_MSEC) {
3220 VLOG_WARN_RL(&rl, "ignoring vote request received after only "
3221 "%lld ms (minimum election time is %d ms)",
3222 now - raft->election_base, ELECTION_BASE_MSEC);
3223 return true;
3224 }
3225 return false;
3226
3227 case RAFT_CANDIDATE:
3228 return false;
3229
3230 default:
3231 OVS_NOT_REACHED();
3232 }
3233}
3234
3235/* Returns true if a reply should be sent. */
3236static bool
3237raft_handle_vote_request__(struct raft *raft,
3238 const struct raft_vote_request *rq)
3239{
3240 /* Figure 3.1: "If votedFor is null or candidateId, and candidate's vote is
3241 * at least as up-to-date as receiver's log, grant vote (sections 3.4,
3242 * 3.6)." */
3243 if (uuid_equals(&raft->vote, &rq->common.sid)) {
3244 /* Already voted for this candidate in this term. Resend vote. */
3245 return true;
3246 } else if (!uuid_is_zero(&raft->vote)) {
3247 /* Already voted for different candidate in this term. Send a reply
3248 * saying what candidate we did vote for. This isn't a necessary part
3249 * of the Raft protocol but it can make debugging easier. */
3250 return true;
3251 }
3252
3253 /* Section 3.6.1: "The RequestVote RPC implements this restriction: the RPC
3254 * includes information about the candidate’s log, and the voter denies its
3255 * vote if its own log is more up-to-date than that of the candidate. Raft
3256 * determines which of two logs is more up-to-date by comparing the index
3257 * and term of the last entries in the logs. If the logs have last entries
3258 * with different terms, then the log with the later term is more
3259 * up-to-date. If the logs end with the same term, then whichever log is
3260 * longer is more up-to-date." */
3261 uint64_t last_term = (raft->log_end > raft->log_start
3262 ? raft->entries[raft->log_end - 1
3263 - raft->log_start].term
3264 : raft->snap.term);
3265 if (last_term > rq->last_log_term
3266 || (last_term == rq->last_log_term
3267 && raft->log_end - 1 > rq->last_log_index)) {
3268 /* Our log is more up-to-date than the peer's. Withhold vote. */
3269 return false;
3270 }
3271
3272 /* Record a vote for the peer. */
3273 if (!raft_set_term(raft, raft->term, &rq->common.sid)) {
3274 return false;
3275 }
3276
3277 raft_reset_timer(raft);
3278
3279 return true;
3280}
3281
3282static void
3283raft_send_vote_reply(struct raft *raft, const struct uuid *dst,
3284 const struct uuid *vote)
3285{
3286 union raft_rpc rpy = {
3287 .vote_reply = {
3288 .common = {
3289 .type = RAFT_RPC_VOTE_REPLY,
3290 .sid = *dst,
3291 },
3292 .term = raft->term,
3293 .vote = *vote,
3294 },
3295 };
3296 raft_send(raft, &rpy);
3297}
3298
3299static void
3300raft_handle_vote_request(struct raft *raft,
3301 const struct raft_vote_request *rq)
3302{
3303 if (raft_handle_vote_request__(raft, rq)) {
3304 raft_send_vote_reply(raft, &rq->common.sid, &raft->vote);
3305 }
3306}
3307
3308static void
3309raft_handle_vote_reply(struct raft *raft,
3310 const struct raft_vote_reply *rpy)
3311{
3312 if (!raft_receive_term__(raft, &rpy->common, rpy->term)) {
3313 return;
3314 }
3315
3316 if (raft->role != RAFT_CANDIDATE) {
3317 return;
3318 }
3319
3320 struct raft_server *s = raft_find_peer(raft, &rpy->common.sid);
3321 if (s) {
3322 raft_accept_vote(raft, s, &rpy->vote);
3323 }
3324}
3325
3326/* Returns true if 'raft''s log contains reconfiguration entries that have not
3327 * yet been committed. */
3328static bool
3329raft_has_uncommitted_configuration(const struct raft *raft)
3330{
3331 for (uint64_t i = raft->commit_index + 1; i < raft->log_end; i++) {
3332 ovs_assert(i >= raft->log_start);
3333 const struct raft_entry *e = &raft->entries[i - raft->log_start];
3334 if (e->servers) {
3335 return true;
3336 }
3337 }
3338 return false;
3339}
3340
3341static void
3342raft_log_reconfiguration(struct raft *raft)
3343{
3344 struct json *servers_json = raft_servers_to_json(&raft->servers);
3345 raft_command_unref(raft_command_execute__(
3346 raft, NULL, servers_json, NULL, NULL));
3347 json_destroy(servers_json);
3348}
3349
3350static void
3351raft_run_reconfigure(struct raft *raft)
3352{
3353 ovs_assert(raft->role == RAFT_LEADER);
3354
3355 /* Reconfiguration only progresses when configuration changes commit. */
3356 if (raft_has_uncommitted_configuration(raft)) {
3357 return;
3358 }
3359
3360 /* If we were waiting for a configuration change to commit, it's done. */
3361 struct raft_server *s;
3362 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
3363 if (s->phase == RAFT_PHASE_COMMITTING) {
3364 raft_send_add_server_reply__(raft, &s->sid, s->address,
3365 true, RAFT_SERVER_COMPLETED);
3366 s->phase = RAFT_PHASE_STABLE;
3367 }
3368 }
3369 if (raft->remove_server) {
3370 raft_send_remove_server_reply__(raft, &raft->remove_server->sid,
3371 &raft->remove_server->requester_sid,
3372 raft->remove_server->requester_conn,
3373 true, RAFT_SERVER_COMPLETED);
3374 raft_server_destroy(raft->remove_server);
3375 raft->remove_server = NULL;
3376 }
3377
3378 /* If a new server is caught up, add it to the configuration. */
3379 HMAP_FOR_EACH (s, hmap_node, &raft->add_servers) {
3380 if (s->phase == RAFT_PHASE_CAUGHT_UP) {
3381 /* Move 's' from 'raft->add_servers' to 'raft->servers'. */
3382 hmap_remove(&raft->add_servers, &s->hmap_node);
3383 hmap_insert(&raft->servers, &s->hmap_node, uuid_hash(&s->sid));
3384
3385 /* Mark 's' as waiting for commit. */
3386 s->phase = RAFT_PHASE_COMMITTING;
3387
3388 raft_log_reconfiguration(raft);
3389
3390 /* When commit completes we'll transition to RAFT_PHASE_STABLE and
3391 * send a RAFT_SERVER_OK reply. */
3392
3393 return;
3394 }
3395 }
3396
3397 /* Remove a server, if one is scheduled for removal. */
3398 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
3399 if (s->phase == RAFT_PHASE_REMOVE) {
3400 hmap_remove(&raft->servers, &s->hmap_node);
3401 raft->remove_server = s;
3402
3403 raft_log_reconfiguration(raft);
3404
3405 return;
3406 }
3407 }
3408}
3409
3410static void
3411raft_handle_add_server_request(struct raft *raft,
3412 const struct raft_add_server_request *rq)
3413{
3414 /* Figure 4.1: "1. Reply NOT_LEADER if not leader (section 6.2)." */
3415 if (raft->role != RAFT_LEADER) {
3416 raft_send_add_server_reply(raft, rq, false, RAFT_SERVER_NOT_LEADER);
3417 return;
3418 }
3419
3420 /* Check for an existing server. */
3421 struct raft_server *s = raft_find_server(raft, &rq->common.sid);
3422 if (s) {
3423 /* If the server is scheduled to be removed, cancel it. */
3424 if (s->phase == RAFT_PHASE_REMOVE) {
3425 s->phase = RAFT_PHASE_STABLE;
3426 raft_send_add_server_reply(raft, rq, false, RAFT_SERVER_CANCELED);
3427 return;
3428 }
3429
3430 /* If the server is being added, then it's in progress. */
3431 if (s->phase != RAFT_PHASE_STABLE) {
3432 raft_send_add_server_reply(raft, rq,
3433 false, RAFT_SERVER_IN_PROGRESS);
3434 }
3435
3436 /* Nothing to do--server is already part of the configuration. */
3437 raft_send_add_server_reply(raft, rq,
3438 true, RAFT_SERVER_ALREADY_PRESENT);
3439 return;
3440 }
3441
3442 /* Check for a server being removed. */
3443 if (raft->remove_server
3444 && uuid_equals(&rq->common.sid, &raft->remove_server->sid)) {
3445 raft_send_add_server_reply(raft, rq, false, RAFT_SERVER_COMMITTING);
3446 return;
3447 }
3448
3449 /* Check for a server already being added. */
3450 if (raft_find_new_server(raft, &rq->common.sid)) {
3451 raft_send_add_server_reply(raft, rq, false, RAFT_SERVER_IN_PROGRESS);
3452 return;
3453 }
3454
3455 /* Add server to 'add_servers'. */
3456 s = raft_server_add(&raft->add_servers, &rq->common.sid, rq->address);
3457 raft_server_init_leader(raft, s);
3458 s->requester_sid = rq->common.sid;
3459 s->requester_conn = NULL;
3460 s->phase = RAFT_PHASE_CATCHUP;
3461
3462 /* Start sending the log. If this is the first time we've tried to add
3463 * this server, then this will quickly degenerate into an InstallSnapshot
3464 * followed by a series of AddEntries, but if it's a retry of an earlier
3465 * AddRequest that was interrupted (e.g. by a timeout or a loss of
3466 * leadership) then it will gracefully resume populating the log.
3467 *
3468 * See the last few paragraphs of section 4.2.1 for further insight. */
3469 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(10, 10);
3470 VLOG_INFO_RL(&rl,
3471 "starting to add server %s ("SID_FMT" at %s) "
3472 "to cluster "CID_FMT, s->nickname, SID_ARGS(&s->sid),
3473 rq->address, CID_ARGS(&raft->cid));
3474 raft_send_append_request(raft, s, 0, "initialize new server");
3475}
3476
3477static void
3478raft_handle_add_server_reply(struct raft *raft,
3479 const struct raft_add_server_reply *rpy)
3480{
3481 if (!raft->joining) {
3482 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
3483 VLOG_WARN_RL(&rl, "received add_server_reply even though we're "
3484 "already part of the cluster");
3485 return;
3486 }
3487
3488 if (rpy->success) {
3489 raft->joining = false;
3490
3491 /* It is tempting, at this point, to check that this server is part of
3492 * the current configuration. However, this is not necessarily the
3493 * case, because the log entry that added this server to the cluster
3494 * might have been committed by a majority of the cluster that does not
3495 * include this one. This actually happens in testing. */
3496 } else {
3497 const char *address;
3498 SSET_FOR_EACH (address, &rpy->remote_addresses) {
3499 if (sset_add(&raft->remote_addresses, address)) {
3500 VLOG_INFO("%s: learned new server address for joining cluster",
3501 address);
3502 }
3503 }
3504 }
3505}
3506
3507/* This is called by raft_unixctl_kick() as well as via RPC. */
3508static void
3509raft_handle_remove_server_request(struct raft *raft,
3510 const struct raft_remove_server_request *rq)
3511{
3512 /* Figure 4.1: "1. Reply NOT_LEADER if not leader (section 6.2)." */
3513 if (raft->role != RAFT_LEADER) {
3514 raft_send_remove_server_reply(raft, rq, false, RAFT_SERVER_NOT_LEADER);
3515 return;
3516 }
3517
3518 /* If the server to remove is currently waiting to be added, cancel it. */
3519 struct raft_server *target = raft_find_new_server(raft, &rq->sid);
3520 if (target) {
3521 raft_send_add_server_reply__(raft, &target->sid, target->address,
3522 false, RAFT_SERVER_CANCELED);
3523 hmap_remove(&raft->add_servers, &target->hmap_node);
3524 raft_server_destroy(target);
3525 return;
3526 }
3527
3528 /* If the server isn't configured, report that. */
3529 target = raft_find_server(raft, &rq->sid);
3530 if (!target) {
3531 raft_send_remove_server_reply(raft, rq,
3532 true, RAFT_SERVER_ALREADY_GONE);
3533 return;
3534 }
3535
3536 /* Check whether we're waiting for the addition of the server to commit. */
3537 if (target->phase == RAFT_PHASE_COMMITTING) {
3538 raft_send_remove_server_reply(raft, rq, false, RAFT_SERVER_COMMITTING);
3539 return;
3540 }
3541
3542 /* Check whether the server is already scheduled for removal. */
3543 if (target->phase == RAFT_PHASE_REMOVE) {
3544 raft_send_remove_server_reply(raft, rq,
3545 false, RAFT_SERVER_IN_PROGRESS);
3546 return;
3547 }
3548
3549 /* Make sure that if we remove this server then that at least one other
3550 * server will be left. We don't count servers currently being added (in
3551 * 'add_servers') since those could fail. */
3552 struct raft_server *s;
3553 int n = 0;
3554 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
3555 if (s != target && s->phase != RAFT_PHASE_REMOVE) {
3556 n++;
3557 }
3558 }
3559 if (!n) {
3560 raft_send_remove_server_reply(raft, rq, false, RAFT_SERVER_EMPTY);
3561 return;
3562 }
3563
3564 /* Mark the server for removal. */
3565 target->phase = RAFT_PHASE_REMOVE;
3566 if (rq->requester_conn) {
3567 target->requester_sid = UUID_ZERO;
3568 unixctl_command_reply(rq->requester_conn, "started removal");
3569 } else {
3570 target->requester_sid = rq->common.sid;
3571 target->requester_conn = NULL;
3572 }
3573
3574 raft_run_reconfigure(raft);
3575 /* Operation in progress, reply will be sent later. */
3576}
3577
3578static void
17bd4149 3579raft_finished_leaving_cluster(struct raft *raft)
1b1d2e6d 3580{
17bd4149
BP
3581 VLOG_INFO(SID_FMT": finished leaving cluster "CID_FMT,
3582 SID_ARGS(&raft->sid), CID_ARGS(&raft->cid));
1b1d2e6d 3583
17bd4149
BP
3584 raft_record_note(raft, "left", "this server left the cluster");
3585
3586 raft->leaving = false;
3587 raft->left = true;
3588}
1b1d2e6d 3589
17bd4149
BP
3590static void
3591raft_handle_remove_server_reply(struct raft *raft,
3592 const struct raft_remove_server_reply *rpc)
3593{
3594 if (rpc->success
3595 && (uuid_is_zero(&rpc->target_sid)
3596 || uuid_equals(&rpc->target_sid, &raft->sid))) {
3597 raft_finished_leaving_cluster(raft);
1b1d2e6d
BP
3598 }
3599}
3600
3601static bool
3602raft_handle_write_error(struct raft *raft, struct ovsdb_error *error)
3603{
3604 if (error && !raft->failed) {
3605 raft->failed = true;
3606
3607 char *s = ovsdb_error_to_string_free(error);
3608 VLOG_WARN("%s: entering failure mode due to I/O error (%s)",
3609 raft->name, s);
3610 free(s);
3611 }
3612 return !raft->failed;
3613}
3614
3615static struct ovsdb_error * OVS_WARN_UNUSED_RESULT
3616raft_write_snapshot(struct raft *raft, struct ovsdb_log *log,
3617 uint64_t new_log_start,
3618 const struct raft_entry *new_snapshot)
3619{
3620 struct raft_header h = {
3621 .sid = raft->sid,
3622 .cid = raft->cid,
3623 .name = raft->name,
3624 .local_address = raft->local_address,
3625 .snap_index = new_log_start - 1,
3626 .snap = *new_snapshot,
3627 };
3628 struct ovsdb_error *error = ovsdb_log_write_and_free(
3629 log, raft_header_to_json(&h));
3630 if (error) {
3631 return error;
3632 }
3633 ovsdb_log_mark_base(raft->log);
3634
3635 /* Write log records. */
3636 for (uint64_t index = new_log_start; index < raft->log_end; index++) {
3637 const struct raft_entry *e = &raft->entries[index - raft->log_start];
3638 struct raft_record r = {
3639 .type = RAFT_REC_ENTRY,
3640 .term = e->term,
3641 .entry = {
3642 .index = index,
3643 .data = e->data,
3644 .servers = e->servers,
3645 .eid = e->eid,
3646 },
3647 };
3648 error = ovsdb_log_write_and_free(log, raft_record_to_json(&r));
3649 if (error) {
3650 return error;
3651 }
3652 }
3653
3654 /* Write term and vote (if any).
3655 *
3656 * The term is redundant if we wrote a log record for that term above. The
3657 * vote, if any, is never redundant.
3658 */
3659 error = raft_write_state(log, raft->term, &raft->vote);
3660 if (error) {
3661 return error;
3662 }
3663
3664 /* Write commit_index if it's beyond the new start of the log. */
3665 if (raft->commit_index >= new_log_start) {
3666 struct raft_record r = {
3667 .type = RAFT_REC_COMMIT_INDEX,
3668 .commit_index = raft->commit_index,
3669 };
3670 return ovsdb_log_write_and_free(log, raft_record_to_json(&r));
3671 }
3672 return NULL;
3673}
3674
3675static struct ovsdb_error * OVS_WARN_UNUSED_RESULT
3676raft_save_snapshot(struct raft *raft,
3677 uint64_t new_start, const struct raft_entry *new_snapshot)
3678
3679{
3680 struct ovsdb_log *new_log;
3681 struct ovsdb_error *error;
3682 error = ovsdb_log_replace_start(raft->log, &new_log);
3683 if (error) {
3684 return error;
3685 }
3686
3687 error = raft_write_snapshot(raft, new_log, new_start, new_snapshot);
3688 if (error) {
3689 ovsdb_log_replace_abort(new_log);
3690 return error;
3691 }
3692
3693 return ovsdb_log_replace_commit(raft->log, new_log);
3694}
3695
3696static bool
3697raft_handle_install_snapshot_request__(
3698 struct raft *raft, const struct raft_install_snapshot_request *rq)
3699{
3700 raft_reset_timer(raft);
3701
3702 /*
3703 * Our behavior here depend on new_log_start in the snapshot compared to
3704 * log_start and log_end. There are three cases:
3705 *
3706 * Case 1 | Case 2 | Case 3
3707 * <---------------->|<------------->|<------------------>
3708 * | |
3709 *
3710 * +---+---+---+---+
3711 * T | T | T | T | T |
3712 * +---+---+---+---+
3713 * ^ ^
3714 * | |
3715 * log_start log_end
3716 */
3717 uint64_t new_log_start = rq->last_index + 1;
3718 if (new_log_start < raft->log_start) {
3719 /* Case 1: The new snapshot covers less than our current one. Nothing
3720 * to do. */
3721 return true;
3722 } else if (new_log_start < raft->log_end) {
3723 /* Case 2: The new snapshot starts in the middle of our log. We could
3724 * discard the first 'new_log_start - raft->log_start' entries in the
3725 * log. But there's not much value in that, since snapshotting is
3726 * supposed to be a local decision. Just skip it. */
3727 return true;
3728 }
3729
3730 /* Case 3: The new snapshot starts past the end of our current log, so
3731 * discard all of our current log. */
3732 const struct raft_entry new_snapshot = {
3733 .term = rq->last_term,
3734 .data = rq->data,
3735 .eid = rq->last_eid,
3736 .servers = rq->last_servers,
3737 };
3738 struct ovsdb_error *error = raft_save_snapshot(raft, new_log_start,
3739 &new_snapshot);
3740 if (error) {
3741 char *error_s = ovsdb_error_to_string(error);
3742 VLOG_WARN("could not save snapshot: %s", error_s);
3743 free(error_s);
3744 return false;
3745 }
3746
3747 for (size_t i = 0; i < raft->log_end - raft->log_start; i++) {
3748 raft_entry_uninit(&raft->entries[i]);
3749 }
3750 raft->log_start = raft->log_end = new_log_start;
3751 raft->log_synced = raft->log_end - 1;
3752 raft->commit_index = raft->log_start - 1;
3753 if (raft->last_applied < raft->commit_index) {
3754 raft->last_applied = raft->log_start - 2;
3755 }
3756
3757 raft_entry_uninit(&raft->snap);
3758 raft_entry_clone(&raft->snap, &new_snapshot);
3759
3760 raft_get_servers_from_log(raft, VLL_INFO);
3761
3762 return true;
3763}
3764
3765static void
3766raft_handle_install_snapshot_request(
3767 struct raft *raft, const struct raft_install_snapshot_request *rq)
3768{
3769 if (raft_handle_install_snapshot_request__(raft, rq)) {
3770 union raft_rpc rpy = {
3771 .install_snapshot_reply = {
3772 .common = {
3773 .type = RAFT_RPC_INSTALL_SNAPSHOT_REPLY,
3774 .sid = rq->common.sid,
3775 },
3776 .term = raft->term,
3777 .last_index = rq->last_index,
3778 .last_term = rq->last_term,
3779 },
3780 };
3781 raft_send(raft, &rpy);
3782 }
3783}
3784
3785static void
3786raft_handle_install_snapshot_reply(
3787 struct raft *raft, const struct raft_install_snapshot_reply *rpy)
3788{
3789 /* We might get an InstallSnapshot reply from a configured server (e.g. a
3790 * peer) or a server in the process of being added. */
3791 struct raft_server *s = raft_find_peer(raft, &rpy->common.sid);
3792 if (!s) {
3793 s = raft_find_new_server(raft, &rpy->common.sid);
3794 if (!s) {
3795 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
3796 VLOG_INFO_RL(&rl, "cluster "CID_FMT": received %s from "
3797 "unknown server "SID_FMT, CID_ARGS(&raft->cid),
3798 raft_rpc_type_to_string(rpy->common.type),
3799 SID_ARGS(&rpy->common.sid));
3800 return;
3801 }
3802 }
3803
3804 if (rpy->last_index != raft->log_start - 1 ||
3805 rpy->last_term != raft->snap.term) {
3806 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
3807 VLOG_INFO_RL(&rl, "cluster "CID_FMT": server %s installed "
3808 "out-of-date snapshot, starting over",
3809 CID_ARGS(&raft->cid), s->nickname);
3810 raft_send_install_snapshot_request(raft, s,
3811 "installed obsolete snapshot");
3812 return;
3813 }
3814
3815 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(10, 10);
3816 VLOG_INFO_RL(&rl, "cluster "CID_FMT": installed snapshot on server %s "
3817 " up to %"PRIu64":%"PRIu64, CID_ARGS(&raft->cid),
3818 s->nickname, rpy->last_term, rpy->last_index);
3819 s->next_index = raft->log_end;
3820 raft_send_append_request(raft, s, 0, "snapshot installed");
3821}
3822
3823/* Returns true if 'raft' has grown enough since the last snapshot that
3824 * reducing the log to a snapshot would be valuable, false otherwise. */
3825bool
3826raft_grew_lots(const struct raft *raft)
3827{
3828 return ovsdb_log_grew_lots(raft->log);
3829}
3830
3831/* Returns the number of log entries that could be trimmed off the on-disk log
3832 * by snapshotting. */
3833uint64_t
3834raft_get_log_length(const struct raft *raft)
3835{
3836 return (raft->last_applied < raft->log_start
3837 ? 0
3838 : raft->last_applied - raft->log_start + 1);
3839}
3840
3841/* Returns true if taking a snapshot of 'raft', with raft_store_snapshot(), is
3842 * possible. */
3843bool
3844raft_may_snapshot(const struct raft *raft)
3845{
3846 return (!raft->joining
3847 && !raft->leaving
3848 && !raft->left
3849 && !raft->failed
3850 && raft->last_applied >= raft->log_start);
3851}
3852
3853/* Replaces the log for 'raft', up to the last log entry read, by
3854 * 'new_snapshot_data'. Returns NULL if successful, otherwise an error that
3855 * the caller must eventually free.
3856 *
3857 * This function can only succeed if raft_may_snapshot() returns true. It is
3858 * only valuable to call it if raft_get_log_length() is significant and
3859 * especially if raft_grew_lots() returns true. */
3860struct ovsdb_error * OVS_WARN_UNUSED_RESULT
3861raft_store_snapshot(struct raft *raft, const struct json *new_snapshot_data)
3862{
3863 if (raft->joining) {
3864 return ovsdb_error(NULL,
3865 "cannot store a snapshot while joining cluster");
3866 } else if (raft->leaving) {
3867 return ovsdb_error(NULL,
3868 "cannot store a snapshot while leaving cluster");
3869 } else if (raft->left) {
3870 return ovsdb_error(NULL,
3871 "cannot store a snapshot after leaving cluster");
3872 } else if (raft->failed) {
3873 return ovsdb_error(NULL,
3874 "cannot store a snapshot following failure");
3875 }
3876
3877 if (raft->last_applied < raft->log_start) {
3878 return ovsdb_error(NULL, "not storing a duplicate snapshot");
3879 }
3880
3881 uint64_t new_log_start = raft->last_applied + 1;
a521491b 3882 struct raft_entry new_snapshot = {
1b1d2e6d 3883 .term = raft_get_term(raft, new_log_start - 1),
a521491b 3884 .data = json_clone(new_snapshot_data),
1b1d2e6d 3885 .eid = *raft_get_eid(raft, new_log_start - 1),
a521491b 3886 .servers = json_clone(raft_servers_for_index(raft, new_log_start - 1)),
1b1d2e6d
BP
3887 };
3888 struct ovsdb_error *error = raft_save_snapshot(raft, new_log_start,
3889 &new_snapshot);
3890 if (error) {
a521491b 3891 raft_entry_uninit(&new_snapshot);
1b1d2e6d
BP
3892 return error;
3893 }
3894
3895 raft->log_synced = raft->log_end - 1;
3896 raft_entry_uninit(&raft->snap);
a521491b 3897 raft->snap = new_snapshot;
1b1d2e6d
BP
3898 for (size_t i = 0; i < new_log_start - raft->log_start; i++) {
3899 raft_entry_uninit(&raft->entries[i]);
3900 }
3901 memmove(&raft->entries[0], &raft->entries[new_log_start - raft->log_start],
3902 (raft->log_end - new_log_start) * sizeof *raft->entries);
3903 raft->log_start = new_log_start;
3904 return NULL;
3905}
3906
3907static void
3908raft_handle_become_leader(struct raft *raft,
3909 const struct raft_become_leader *rq)
3910{
3911 if (raft->role == RAFT_FOLLOWER) {
3912 char buf[SID_LEN + 1];
3913 VLOG_INFO("received leadership transfer from %s in term %"PRIu64,
3914 raft_get_nickname(raft, &rq->common.sid, buf, sizeof buf),
3915 rq->term);
3916 raft_start_election(raft, true);
3917 }
3918}
3919
3920static void
3921raft_send_execute_command_reply(struct raft *raft,
3922 const struct uuid *sid,
3923 const struct uuid *eid,
3924 enum raft_command_status status,
3925 uint64_t commit_index)
3926{
3927 union raft_rpc rpc = {
3928 .execute_command_reply = {
3929 .common = {
3930 .type = RAFT_RPC_EXECUTE_COMMAND_REPLY,
3931 .sid = *sid,
3932 },
3933 .result = *eid,
3934 .status = status,
3935 .commit_index = commit_index,
3936 },
3937 };
3938 raft_send(raft, &rpc);
3939}
3940
3941static enum raft_command_status
3942raft_handle_execute_command_request__(
3943 struct raft *raft, const struct raft_execute_command_request *rq)
3944{
3945 if (raft->role != RAFT_LEADER) {
3946 return RAFT_CMD_NOT_LEADER;
3947 }
3948
3949 const struct uuid *current_eid = raft_current_eid(raft);
3950 if (!uuid_equals(&rq->prereq, current_eid)) {
3951 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
3952 VLOG_INFO_RL(&rl, "current entry eid "UUID_FMT" does not match "
3953 "prerequisite "UUID_FMT" in execute_command_request",
3954 UUID_ARGS(current_eid), UUID_ARGS(&rq->prereq));
3955 return RAFT_CMD_BAD_PREREQ;
3956 }
3957
3958 struct raft_command *cmd = raft_command_initiate(raft, rq->data,
3959 NULL, &rq->result);
3960 cmd->sid = rq->common.sid;
3961
3962 enum raft_command_status status = cmd->status;
3963 if (status != RAFT_CMD_INCOMPLETE) {
3964 raft_command_unref(cmd);
3965 }
3966 return status;
3967}
3968
3969static void
3970raft_handle_execute_command_request(
3971 struct raft *raft, const struct raft_execute_command_request *rq)
3972{
3973 enum raft_command_status status
3974 = raft_handle_execute_command_request__(raft, rq);
3975 if (status != RAFT_CMD_INCOMPLETE) {
3976 raft_send_execute_command_reply(raft, &rq->common.sid, &rq->result,
3977 status, 0);
3978 }
3979}
3980
3981static void
3982raft_handle_execute_command_reply(
3983 struct raft *raft, const struct raft_execute_command_reply *rpy)
3984{
3985 struct raft_command *cmd = raft_find_command_by_eid(raft, &rpy->result);
3986 if (!cmd) {
3987 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
3988 char buf[SID_LEN + 1];
3989 VLOG_INFO_RL(&rl,
3990 "%s received \"%s\" reply from %s for unknown command",
3991 raft->local_nickname,
3992 raft_command_status_to_string(rpy->status),
3993 raft_get_nickname(raft, &rpy->common.sid,
3994 buf, sizeof buf));
3995 return;
3996 }
3997
3998 if (rpy->status == RAFT_CMD_INCOMPLETE) {
3999 cmd->timestamp = time_msec();
4000 } else {
4001 cmd->index = rpy->commit_index;
4002 raft_command_complete(raft, cmd, rpy->status);
4003 }
4004}
4005
4006static void
4007raft_handle_rpc(struct raft *raft, const union raft_rpc *rpc)
4008{
4009 uint64_t term = raft_rpc_get_term(rpc);
4010 if (term
4011 && !raft_should_suppress_disruptive_server(raft, rpc)
4012 && !raft_receive_term__(raft, &rpc->common, term)) {
4013 if (rpc->type == RAFT_RPC_APPEND_REQUEST) {
4014 /* Section 3.3: "If a server receives a request with a stale term
4015 * number, it rejects the request." */
4016 raft_send_append_reply(raft, raft_append_request_cast(rpc),
4017 RAFT_APPEND_INCONSISTENCY, "stale term");
4018 }
4019 return;
4020 }
4021
4022 switch (rpc->type) {
4023#define RAFT_RPC(ENUM, NAME) \
4024 case ENUM: \
4025 raft_handle_##NAME(raft, &rpc->NAME); \
4026 break;
4027 RAFT_RPC_TYPES
4028#undef RAFT_RPC
4029 default:
4030 OVS_NOT_REACHED();
4031 }
4032}
4033\f
4034static bool
4035raft_rpc_is_heartbeat(const union raft_rpc *rpc)
4036{
4037 return ((rpc->type == RAFT_RPC_APPEND_REQUEST
4038 || rpc->type == RAFT_RPC_APPEND_REPLY)
4039 && rpc->common.comment
4040 && !strcmp(rpc->common.comment, "heartbeat"));
4041}
4042
4043\f
4044static bool
02acb41a
BP
4045raft_send_to_conn_at(struct raft *raft, const union raft_rpc *rpc,
4046 struct raft_conn *conn, int line_number)
1b1d2e6d 4047{
02acb41a 4048 log_rpc(rpc, "-->", conn, line_number);
1b1d2e6d
BP
4049 return !jsonrpc_session_send(
4050 conn->js, raft_rpc_to_jsonrpc(&raft->cid, &raft->sid, rpc));
4051}
4052
4053static bool
4054raft_is_rpc_synced(const struct raft *raft, const union raft_rpc *rpc)
4055{
4056 uint64_t term = raft_rpc_get_term(rpc);
4057 uint64_t index = raft_rpc_get_min_sync_index(rpc);
4058 const struct uuid *vote = raft_rpc_get_vote(rpc);
4059
4060 return (term <= raft->synced_term
4061 && index <= raft->log_synced
4062 && (!vote || uuid_equals(vote, &raft->synced_vote)));
4063}
4064
4065static bool
02acb41a 4066raft_send_at(struct raft *raft, const union raft_rpc *rpc, int line_number)
1b1d2e6d
BP
4067{
4068 const struct uuid *dst = &rpc->common.sid;
4069 if (uuid_equals(dst, &raft->sid)) {
4070 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
02acb41a
BP
4071 VLOG_WARN_RL(&rl, "attempted to send RPC to self from raft.c:%d",
4072 line_number);
1b1d2e6d
BP
4073 return false;
4074 }
4075
4076 struct raft_conn *conn = raft_find_conn_by_sid(raft, dst);
4077 if (!conn) {
4078 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
4079 char buf[SID_LEN + 1];
02acb41a
BP
4080 VLOG_DBG_RL(&rl, "%s: no connection to %s, cannot send RPC "
4081 "from raft.c:%d", raft->local_nickname,
4082 raft_get_nickname(raft, dst, buf, sizeof buf),
4083 line_number);
1b1d2e6d
BP
4084 return false;
4085 }
4086
4087 if (!raft_is_rpc_synced(raft, rpc)) {
4088 raft_waiter_create(raft, RAFT_W_RPC, false)->rpc = raft_rpc_clone(rpc);
4089 return true;
4090 }
4091
02acb41a 4092 return raft_send_to_conn_at(raft, rpc, conn, line_number);
1b1d2e6d
BP
4093}
4094\f
4095static struct raft *
4096raft_lookup_by_name(const char *name)
4097{
4098 struct raft *raft;
4099
4100 HMAP_FOR_EACH_WITH_HASH (raft, hmap_node, hash_string(name, 0),
4101 &all_rafts) {
4102 if (!strcmp(raft->name, name)) {
4103 return raft;
4104 }
4105 }
4106 return NULL;
4107}
4108
4109static void
4110raft_unixctl_cid(struct unixctl_conn *conn,
4111 int argc OVS_UNUSED, const char *argv[],
4112 void *aux OVS_UNUSED)
4113{
4114 struct raft *raft = raft_lookup_by_name(argv[1]);
4115 if (!raft) {
4116 unixctl_command_reply_error(conn, "unknown cluster");
4117 } else if (uuid_is_zero(&raft->cid)) {
4118 unixctl_command_reply_error(conn, "cluster id not yet known");
4119 } else {
4120 char *uuid = xasprintf(UUID_FMT, UUID_ARGS(&raft->cid));
4121 unixctl_command_reply(conn, uuid);
4122 free(uuid);
4123 }
4124}
4125
4126static void
4127raft_unixctl_sid(struct unixctl_conn *conn,
4128 int argc OVS_UNUSED, const char *argv[],
4129 void *aux OVS_UNUSED)
4130{
4131 struct raft *raft = raft_lookup_by_name(argv[1]);
4132 if (!raft) {
4133 unixctl_command_reply_error(conn, "unknown cluster");
4134 } else {
4135 char *uuid = xasprintf(UUID_FMT, UUID_ARGS(&raft->sid));
4136 unixctl_command_reply(conn, uuid);
4137 free(uuid);
4138 }
4139}
4140
4141static void
4142raft_put_sid(const char *title, const struct uuid *sid,
4143 const struct raft *raft, struct ds *s)
4144{
4145 ds_put_format(s, "%s: ", title);
4146 if (uuid_equals(sid, &raft->sid)) {
4147 ds_put_cstr(s, "self");
4148 } else if (uuid_is_zero(sid)) {
4149 ds_put_cstr(s, "unknown");
4150 } else {
4151 char buf[SID_LEN + 1];
4152 ds_put_cstr(s, raft_get_nickname(raft, sid, buf, sizeof buf));
4153 }
4154 ds_put_char(s, '\n');
4155}
4156
4157static void
4158raft_unixctl_status(struct unixctl_conn *conn,
4159 int argc OVS_UNUSED, const char *argv[],
4160 void *aux OVS_UNUSED)
4161{
4162 struct raft *raft = raft_lookup_by_name(argv[1]);
4163 if (!raft) {
4164 unixctl_command_reply_error(conn, "unknown cluster");
4165 return;
4166 }
4167
4168 struct ds s = DS_EMPTY_INITIALIZER;
4169 ds_put_format(&s, "%s\n", raft->local_nickname);
4170 ds_put_format(&s, "Name: %s\n", raft->name);
4171 ds_put_format(&s, "Cluster ID: ");
4172 if (!uuid_is_zero(&raft->cid)) {
4173 ds_put_format(&s, CID_FMT" ("UUID_FMT")\n",
4174 CID_ARGS(&raft->cid), UUID_ARGS(&raft->cid));
4175 } else {
4176 ds_put_format(&s, "not yet known\n");
4177 }
4178 ds_put_format(&s, "Server ID: "SID_FMT" ("UUID_FMT")\n",
4179 SID_ARGS(&raft->sid), UUID_ARGS(&raft->sid));
4180 ds_put_format(&s, "Address: %s\n", raft->local_address);
4181 ds_put_format(&s, "Status: %s\n",
4182 raft->joining ? "joining cluster"
4183 : raft->leaving ? "leaving cluster"
4184 : raft->left ? "left cluster"
4185 : raft->failed ? "failed"
4186 : "cluster member");
4187 if (raft->joining) {
4188 ds_put_format(&s, "Remotes for joining:");
4189 const char *address;
4190 SSET_FOR_EACH (address, &raft->remote_addresses) {
4191 ds_put_format(&s, " %s", address);
4192 }
4193 ds_put_char(&s, '\n');
4194 }
4195 if (raft->role == RAFT_LEADER) {
4196 struct raft_server *as;
4197 HMAP_FOR_EACH (as, hmap_node, &raft->add_servers) {
4198 ds_put_format(&s, "Adding server %s ("SID_FMT" at %s) (%s)\n",
4199 as->nickname, SID_ARGS(&as->sid), as->address,
4200 raft_server_phase_to_string(as->phase));
4201 }
4202
4203 struct raft_server *rs = raft->remove_server;
4204 if (rs) {
4205 ds_put_format(&s, "Removing server %s ("SID_FMT" at %s) (%s)\n",
4206 rs->nickname, SID_ARGS(&rs->sid), rs->address,
4207 raft_server_phase_to_string(rs->phase));
4208 }
4209 }
4210
4211 ds_put_format(&s, "Role: %s\n",
4212 raft->role == RAFT_LEADER ? "leader"
4213 : raft->role == RAFT_CANDIDATE ? "candidate"
4214 : raft->role == RAFT_FOLLOWER ? "follower"
4215 : "<error>");
4216 ds_put_format(&s, "Term: %"PRIu64"\n", raft->term);
4217 raft_put_sid("Leader", &raft->leader_sid, raft, &s);
4218 raft_put_sid("Vote", &raft->vote, raft, &s);
4219 ds_put_char(&s, '\n');
4220
4221 ds_put_format(&s, "Log: [%"PRIu64", %"PRIu64"]\n",
4222 raft->log_start, raft->log_end);
4223
4224 uint64_t n_uncommitted = raft->log_end - raft->commit_index - 1;
4225 ds_put_format(&s, "Entries not yet committed: %"PRIu64"\n", n_uncommitted);
4226
4227 uint64_t n_unapplied = raft->log_end - raft->last_applied - 1;
4228 ds_put_format(&s, "Entries not yet applied: %"PRIu64"\n", n_unapplied);
4229
4230 const struct raft_conn *c;
4231 ds_put_cstr(&s, "Connections:");
4232 LIST_FOR_EACH (c, list_node, &raft->conns) {
4233 bool connected = jsonrpc_session_is_connected(c->js);
4234 ds_put_format(&s, " %s%s%s%s",
4235 connected ? "" : "(",
4236 c->incoming ? "<-" : "->", c->nickname,
4237 connected ? "" : ")");
4238 }
4239 ds_put_char(&s, '\n');
4240
4241 ds_put_cstr(&s, "Servers:\n");
4242 struct raft_server *server;
4243 HMAP_FOR_EACH (server, hmap_node, &raft->servers) {
4244 ds_put_format(&s, " %s ("SID_FMT" at %s)",
4245 server->nickname,
4246 SID_ARGS(&server->sid), server->address);
4247 if (uuid_equals(&server->sid, &raft->sid)) {
4248 ds_put_cstr(&s, " (self)");
4249 }
4250 if (server->phase != RAFT_PHASE_STABLE) {
4251 ds_put_format (&s, " (%s)",
4252 raft_server_phase_to_string(server->phase));
4253 }
4254 if (raft->role == RAFT_CANDIDATE) {
4255 if (!uuid_is_zero(&server->vote)) {
4256 char buf[SID_LEN + 1];
4257 ds_put_format(&s, " (voted for %s)",
4258 raft_get_nickname(raft, &server->vote,
4259 buf, sizeof buf));
4260 }
4261 } else if (raft->role == RAFT_LEADER) {
4262 ds_put_format(&s, " next_index=%"PRIu64" match_index=%"PRIu64,
4263 server->next_index, server->match_index);
4264 }
4265 ds_put_char(&s, '\n');
4266 }
4267
4268 unixctl_command_reply(conn, ds_cstr(&s));
4269 ds_destroy(&s);
4270}
4271
4272static void
4273raft_unixctl_leave__(struct unixctl_conn *conn, struct raft *raft)
4274{
4275 if (raft_is_leaving(raft)) {
4276 unixctl_command_reply_error(conn,
4277 "already in progress leaving cluster");
4278 } else if (raft_is_joining(raft)) {
4279 unixctl_command_reply_error(conn,
4280 "can't leave while join in progress");
4281 } else if (raft_failed(raft)) {
4282 unixctl_command_reply_error(conn,
4283 "can't leave after failure");
4284 } else {
4285 raft_leave(raft);
4286 unixctl_command_reply(conn, NULL);
4287 }
4288}
4289
4290static void
4291raft_unixctl_leave(struct unixctl_conn *conn, int argc OVS_UNUSED,
4292 const char *argv[], void *aux OVS_UNUSED)
4293{
4294 struct raft *raft = raft_lookup_by_name(argv[1]);
4295 if (!raft) {
4296 unixctl_command_reply_error(conn, "unknown cluster");
4297 return;
4298 }
4299
4300 raft_unixctl_leave__(conn, raft);
4301}
4302
4303static struct raft_server *
4304raft_lookup_server_best_match(struct raft *raft, const char *id)
4305{
4306 struct raft_server *best = NULL;
4307 int best_score = -1;
4308 int n_best = 0;
4309
4310 struct raft_server *s;
4311 HMAP_FOR_EACH (s, hmap_node, &raft->servers) {
4312 int score = (!strcmp(id, s->address)
4313 ? INT_MAX
4314 : uuid_is_partial_match(&s->sid, id));
4315 if (score > best_score) {
4316 best = s;
4317 best_score = score;
4318 n_best = 1;
4319 } else if (score == best_score) {
4320 n_best++;
4321 }
4322 }
4323 return n_best == 1 ? best : NULL;
4324}
4325
4326static void
4327raft_unixctl_kick(struct unixctl_conn *conn, int argc OVS_UNUSED,
4328 const char *argv[], void *aux OVS_UNUSED)
4329{
4330 const char *cluster_name = argv[1];
4331 const char *server_name = argv[2];
4332
4333 struct raft *raft = raft_lookup_by_name(cluster_name);
4334 if (!raft) {
4335 unixctl_command_reply_error(conn, "unknown cluster");
4336 return;
4337 }
4338
4339 struct raft_server *server = raft_lookup_server_best_match(raft,
4340 server_name);
4341 if (!server) {
4342 unixctl_command_reply_error(conn, "unknown server");
4343 return;
4344 }
4345
4346 if (uuid_equals(&server->sid, &raft->sid)) {
4347 raft_unixctl_leave__(conn, raft);
4348 } else if (raft->role == RAFT_LEADER) {
4349 const struct raft_remove_server_request rq = {
4350 .sid = server->sid,
4351 .requester_conn = conn,
4352 };
4353 raft_handle_remove_server_request(raft, &rq);
4354 } else {
4355 const union raft_rpc rpc = {
4356 .remove_server_request = {
4357 .common = {
4358 .type = RAFT_RPC_REMOVE_SERVER_REQUEST,
4359 .sid = raft->leader_sid,
4360 .comment = "via unixctl"
4361 },
4362 .sid = server->sid,
4363 }
4364 };
4365 if (raft_send(raft, &rpc)) {
4366 unixctl_command_reply(conn, "sent removal request to leader");
4367 } else {
4368 unixctl_command_reply_error(conn,
4369 "failed to send removal request");
4370 }
4371 }
4372}
4373
4374static void
4375raft_init(void)
4376{
4377 static struct ovsthread_once once = OVSTHREAD_ONCE_INITIALIZER;
4378 if (!ovsthread_once_start(&once)) {
4379 return;
4380 }
4381 unixctl_command_register("cluster/cid", "DB", 1, 1,
4382 raft_unixctl_cid, NULL);
4383 unixctl_command_register("cluster/sid", "DB", 1, 1,
4384 raft_unixctl_sid, NULL);
4385 unixctl_command_register("cluster/status", "DB", 1, 1,
4386 raft_unixctl_status, NULL);
4387 unixctl_command_register("cluster/leave", "DB", 1, 1,
4388 raft_unixctl_leave, NULL);
4389 unixctl_command_register("cluster/kick", "DB SERVER", 2, 2,
4390 raft_unixctl_kick, NULL);
4391 ovsthread_once_done(&once);
4392}