-
Notifications
You must be signed in to change notification settings - Fork 249
Expand file tree
/
Copy pathraft.h
More file actions
2893 lines (2555 loc) · 86.6 KB
/
raft.h
File metadata and controls
2893 lines (2555 loc) · 86.6 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright (c) Microsoft Corporation. All rights reserved.
// Licensed under the Apache 2.0 License.
#pragma once
#include "ccf/pal/locking.h"
#include "ccf/service/reconfiguration_type.h"
#include "ccf/tx_id.h"
#include "ccf/tx_status.h"
#include "consensus/aft/raft_types.h"
#include "ds/ccf_assert.h"
#include "ds/internal_logger.h"
#include "ds/serialized.h"
#include "impl/state.h"
#include "kv/kv_types.h"
#include "node/node_client.h"
#include "node/node_to_node.h"
#include "node/node_types.h"
#include "node/retired_nodes_cleanup.h"
#include "raft_types.h"
#include "service/tables/signatures.h"
#include <algorithm>
#include <list>
#include <random>
#include <unordered_map>
#include <vector>
#ifdef VERBOSE_RAFT_LOGGING
# define RAFT_TRACE_FMT(s, ...) \
CCF_LOG_FMT(TRACE, "raft") \
("{} | {} | {} | " s, \
state->node_id, \
state->leadership_state, \
state->membership_state, \
##__VA_ARGS__)
# define RAFT_DEBUG_FMT(s, ...) \
CCF_LOG_FMT(DEBUG, "raft") \
("{} | {} | {} | " s, \
state->node_id, \
state->leadership_state, \
state->membership_state, \
##__VA_ARGS__)
# define RAFT_INFO_FMT(s, ...) \
CCF_LOG_FMT(INFO, "raft") \
("{} | {} | {} | " s, \
state->node_id, \
state->leadership_state, \
state->membership_state, \
##__VA_ARGS__)
# define RAFT_FAIL_FMT(s, ...) \
CCF_LOG_FMT(FAIL, "raft") \
("{} | {} | {} | " s, \
state->node_id, \
state->leadership_state, \
state->membership_state, \
##__VA_ARGS__)
#else
# define RAFT_TRACE_FMT LOG_TRACE_FMT
# define RAFT_DEBUG_FMT LOG_DEBUG_FMT
# define RAFT_INFO_FMT LOG_INFO_FMT
# define RAFT_FAIL_FMT LOG_FAIL_FMT
#endif
#define RAFT_TRACE_JSON_OUT(json_object) \
CCF_LOG_OUT(DEBUG, "raft_trace") << json_object
#ifdef CCF_RAFT_TRACING
static inline void add_committable_indices_start_and_end(
nlohmann::json& j, const std::shared_ptr<aft::State>& state)
{
std::vector<aft::Index> committable_indices;
if (!state->committable_indices.empty())
{
committable_indices.push_back(state->committable_indices.front());
if (state->committable_indices.size() > 1)
{
committable_indices.push_back(state->committable_indices.back());
}
}
j["committable_indices"] = committable_indices;
}
# define COMMITTABLE_INDICES(event_state, state) \
add_committable_indices_start_and_end(event_state, state);
#endif
#define LOG_ROLLBACK_INFO_FMT CCF_LOG_FMT(INFO, "rollback")
namespace aft
{
using Configuration = ccf::kv::Configuration;
template <class LedgerProxy>
class Aft : public ccf::kv::Consensus
{
private:
struct NodeState
{
Configuration::NodeInfo node_info;
// the highest index sent to the node
Index sent_idx = 0;
// the highest matching index with the node that was confirmed
Index match_idx = 0;
// timeout tracking the last time an ack was received from the node
std::chrono::milliseconds last_ack_timeout{0};
NodeState() = default;
NodeState(
Configuration::NodeInfo node_info_,
Index sent_idx_,
Index match_idx_ = 0) :
node_info(std::move(node_info_)),
sent_idx(sent_idx_),
match_idx(match_idx_),
last_ack_timeout(0)
{}
};
// Persistent
std::unique_ptr<Store> store;
// Volatile
std::optional<ccf::NodeId> voted_for = std::nullopt;
std::optional<ccf::NodeId> leader_id = std::nullopt;
// Keep track of votes in each active configuration
struct Votes
{
std::unordered_set<ccf::NodeId> votes;
size_t quorum = 0;
};
std::map<Index, Votes> votes_for_me;
std::chrono::milliseconds timeout_elapsed;
// When this node receives append entries from a new primary, it may need to
// roll back a committable but uncommitted suffix it holds. The
// new primary dictates the index where this suffix begins, which
// following the Raft election rules must be at least as high as the highest
// commit index reported by the previous primary. The window in which this
// rollback could be accepted is minimised to avoid unnecessary
// retransmissions - this node only executes this rollback instruction on
// the first append entries after it became a follower. As with any append
// entries, the initial index will not advance until this node acks.
bool is_new_follower = false;
// When this node becomes primary, they should produce a new signature in
// the current view. This signature is the first thing they may commit, as
// they cannot confirm commit of anything from a previous view (Raft paper
// section 5.4.2). This bool is true from the point this node becomes
// primary, until it sees a committable entry
bool should_sign = false;
std::shared_ptr<aft::State> state;
// Timeouts
std::chrono::milliseconds request_timeout;
std::chrono::milliseconds election_timeout;
size_t max_uncommitted_tx_count;
bool ticking = false;
// Configurations
std::list<Configuration> configurations;
// Union of other nodes (i.e. all nodes but us) in each active
// configuration, plus those that are retired, but for which
// the persistence of retirement knowledge is not yet established,
// i.e. Completed but not RetiredCommitted
// This should be used for diagnostic or broadcasting
// messages but _not_ for counting quorums, which should be done for each
// active configuration.
std::unordered_map<ccf::NodeId, NodeState> all_other_nodes;
std::unordered_map<ccf::NodeId, ccf::SeqNo> retired_nodes;
// Node client to trigger submission of RPC requests
std::shared_ptr<ccf::NodeClient> node_client;
// Used to remove retired nodes from store
std::unique_ptr<ccf::RetiredNodeCleanup> retired_node_cleanup;
size_t entry_size_not_limited = 0;
size_t entry_count = 0;
Index entries_batch_size = 20;
static constexpr int batch_window_size = 100;
int batch_window_sum = 0;
// When this is set, only public domain is deserialised when receiving
// append entries
bool public_only = false;
// Randomness
std::uniform_int_distribution<int> distrib;
std::default_random_engine rand;
// AppendEntries messages are currently constrained to only contain entries
// from a single term, so that the receiver can know the term of each entry
// pre-deserialisation, without an additional header.
static constexpr size_t max_terms_per_append_entries = 1;
public:
static constexpr size_t append_entries_size_limit = 20000;
std::unique_ptr<LedgerProxy> ledger;
std::shared_ptr<ccf::NodeToNode> channels;
enum class StartupRole : std::uint8_t
{
Primary,
Backup,
};
// Describes the initial role and state for this node at construction
// time, before any other thread can observe it (so no lock is needed).
struct StartupState
{
StartupRole role;
// State to apply before becoming primary/backup. When nullopt for
// a primary, the node starts from scratch (genesis).
struct StateInfo
{
Index index = 0;
Term term = 0;
std::vector<Index> view_history;
Index recovery_start_index = 0;
};
std::optional<StateInfo> info = std::nullopt;
};
Aft(
const ccf::consensus::Configuration& settings_,
std::unique_ptr<Store> store_,
std::unique_ptr<LedgerProxy> ledger_,
std::shared_ptr<ccf::NodeToNode> channels_,
std::shared_ptr<aft::State> state_,
std::shared_ptr<ccf::NodeClient> rpc_request_context_,
bool public_only_ = false,
std::optional<StartupState> startup = std::nullopt) :
store(std::move(store_)),
timeout_elapsed(0),
state(std::move(state_)),
request_timeout(settings_.message_timeout),
election_timeout(settings_.election_timeout),
max_uncommitted_tx_count(settings_.max_uncommitted_tx_count),
node_client(std::move(rpc_request_context_)),
retired_node_cleanup(
std::make_unique<ccf::RetiredNodeCleanup>(node_client)),
public_only(public_only_),
distrib(0, (int)election_timeout.count() / 2),
rand((int)(uintptr_t)this),
ledger(std::move(ledger_)),
channels(std::move(channels_))
{
if (startup.has_value())
{
const auto& s = startup.value();
if (s.info.has_value())
{
const auto& si = s.info.value();
if (s.role == StartupRole::Primary)
{
state->current_view = si.term;
state->last_idx = si.index;
state->commit_idx = si.index;
state->view_history.initialise(si.view_history);
state->view_history.update(si.index, si.term);
}
else
{
state->last_idx = si.index;
state->commit_idx = si.index;
state->view_history.initialise(si.view_history);
ledger->init(si.index, si.recovery_start_index);
become_aware_of_new_term(si.term);
}
}
if (s.role == StartupRole::Primary)
{
state->current_view += starting_view_change;
become_leader(true);
}
}
}
~Aft() override = default;
std::optional<ccf::NodeId> primary() override
{
return leader_id;
}
ccf::NodeId id() override
{
return state->node_id;
}
bool is_primary() override
{
return state->leadership_state == ccf::kv::LeadershipState::Leader;
}
bool is_candidate() override
{
return state->leadership_state == ccf::kv::LeadershipState::Candidate;
}
bool can_replicate() override
{
std::unique_lock<ccf::pal::Mutex> guard(state->lock);
return can_replicate_unsafe();
}
/**
* Returns true if the node is primary, max_uncommitted_tx_count is non-zero
* and the number of transactions replicated but not yet committed exceeds
* max_uncommitted_tx_count.
*/
bool is_at_max_capacity() override
{
if (max_uncommitted_tx_count == 0)
{
return false;
}
std::unique_lock<ccf::pal::Mutex> guard(state->lock);
return state->leadership_state == ccf::kv::LeadershipState::Leader &&
(state->last_idx - state->commit_idx >= max_uncommitted_tx_count);
}
Consensus::SignatureDisposition get_signature_disposition() override
{
std::unique_lock<ccf::pal::Mutex> guard(state->lock);
if (can_sign_unsafe())
{
if (should_sign)
{
return Consensus::SignatureDisposition::SHOULD_SIGN;
}
return Consensus::SignatureDisposition::CAN_SIGN;
}
return Consensus::SignatureDisposition::CANT_REPLICATE;
}
bool is_backup() override
{
return state->leadership_state == ccf::kv::LeadershipState::Follower;
}
bool is_active() const
{
return state->membership_state == ccf::kv::MembershipState::Active;
}
bool is_retired() const
{
return state->membership_state == ccf::kv::MembershipState::Retired;
}
bool is_retired_committed() const
{
return state->membership_state == ccf::kv::MembershipState::Retired &&
state->retirement_phase == ccf::kv::RetirementPhase::RetiredCommitted;
}
bool is_retired_completed() const
{
return state->membership_state == ccf::kv::MembershipState::Retired &&
state->retirement_phase == ccf::kv::RetirementPhase::Completed;
}
void set_retired_committed(
ccf::SeqNo seqno, const std::vector<ccf::kv::NodeId>& node_ids) override
{
for (const auto& node_id : node_ids)
{
if (id() == node_id)
{
CCF_ASSERT(
state->membership_state == ccf::kv::MembershipState::Retired,
"Node is not retired, cannot become retired committed");
CCF_ASSERT(
state->retirement_phase == ccf::kv::RetirementPhase::Completed,
"Node is not retired, cannot become retired committed");
state->retired_committed_idx = seqno;
become_retired(seqno, ccf::kv::RetirementPhase::RetiredCommitted);
}
else
{
// Once a node's retired_committed status is itself committed, all
// future primaries in the network must be aware its retirement is
// committed, and so no longer need any communication with it to
// advance commit. No further communication with this node is needed.
all_other_nodes.erase(node_id);
RAFT_INFO_FMT("Removed {} from nodes known to consensus", node_id);
}
}
}
Index last_committable_index() const
{
return state->committable_indices.empty() ?
state->commit_idx :
state->committable_indices.back();
}
// Returns the highest committable index which is not greater than the
// given idx.
std::optional<Index> find_highest_possible_committable_index(
Index idx) const
{
const auto it = std::upper_bound(
state->committable_indices.rbegin(),
state->committable_indices.rend(),
idx,
[](const auto& l, const auto& r) { return l >= r; });
if (it == state->committable_indices.rend())
{
return std::nullopt;
}
return *it;
}
void compact_committable_indices(Index idx)
{
while (!state->committable_indices.empty() &&
(state->committable_indices.front() <= idx))
{
state->committable_indices.pop_front();
}
}
void enable_all_domains() override
{
// When receiving append entries as a follower, all security domains will
// be deserialised
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
public_only = false;
}
void force_become_primary() override
{
// This is unsafe and should only be called when the node is certain
// there is no leader and no other node will attempt to force leadership.
if (leader_id.has_value())
{
throw std::logic_error(
"Can't force leadership if there is already a leader");
}
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
state->current_view += starting_view_change;
become_leader(true);
}
void force_become_primary(
Index index,
Term term,
const std::vector<Index>& terms,
Index commit_idx_) override
{
// This is unsafe and should only be called when the node is certain
// there is no leader and no other node will attempt to force leadership.
if (leader_id.has_value())
{
throw std::logic_error(
"Can't force leadership if there is already a leader");
}
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
state->current_view = term;
state->last_idx = index;
state->commit_idx = commit_idx_;
state->view_history.initialise(terms);
state->view_history.update(index, term);
state->current_view += starting_view_change;
become_leader(true);
}
void init_as_backup(
Index index,
Term term,
const std::vector<Index>& term_history,
Index recovery_start_index = 0) override
{
// This should only be called when the node resumes from a snapshot and
// before it has received any append entries.
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
state->last_idx = index;
state->commit_idx = index;
state->view_history.initialise(term_history);
ledger->init(index, recovery_start_index);
become_aware_of_new_term(term);
}
Index get_last_idx()
{
return state->last_idx;
}
Index get_committed_seqno() override
{
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
return get_commit_idx_unsafe();
}
Term get_view() override
{
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
return state->current_view;
}
std::pair<Term, Index> get_committed_txid() override
{
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
ccf::SeqNo commit_idx = get_commit_idx_unsafe();
return {get_term_internal(commit_idx), commit_idx};
}
Term get_view(Index idx) override
{
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
return get_term_internal(idx);
}
std::vector<Index> get_view_history(Index idx) override
{
// This should only be called when the spin lock is held.
return state->view_history.get_history_until(idx);
}
std::vector<Index> get_view_history_since(Index idx) override
{
// This should only be called when the spin lock is held.
return state->view_history.get_history_since(idx);
}
// Same as ccfraft.tla GetServerSet/IsInServerSet
// Not to be confused with all_other_nodes, which includes retired completed
// nodes. Used to restrict sending vote requests, and when becoming a
// leader, to decide whether to advance commit.
std::set<ccf::NodeId> other_nodes_in_active_configs() const
{
std::set<ccf::NodeId> nodes;
for (auto const& conf : configurations)
{
for (auto const& [node_id, _] : conf.nodes)
{
if (node_id != state->node_id)
{
nodes.insert(node_id);
}
}
}
return nodes;
}
void add_configuration(
Index idx, const ccf::kv::Configuration::Nodes& conf) override
{
RAFT_DEBUG_FMT(
"Configurations: add new configuration at {}: {{{}}}", idx, conf);
#ifdef CCF_RAFT_TRACING
nlohmann::json j = {};
j["function"] = "add_configuration";
j["state"] = *state;
COMMITTABLE_INDICES(j["state"], state);
j["configurations"] = configurations;
j["args"] = nlohmann::json::object();
j["args"]["configuration"] = Configuration{idx, conf, idx};
RAFT_TRACE_JSON_OUT(j);
#endif
// Detect when we are retired by observing a configuration
// from which we are absent following a configuration in which
// we were included. Note that this relies on retirement being
// a final state, and node identities never being re-used.
if (
!configurations.empty() &&
configurations.back().nodes.find(state->node_id) !=
configurations.back().nodes.end() &&
conf.find(state->node_id) == conf.end())
{
become_retired(idx, ccf::kv::RetirementPhase::Ordered);
}
if (configurations.empty() || conf != configurations.back().nodes)
{
Configuration new_config = {idx, conf, idx};
configurations.push_back(new_config);
create_and_remove_node_state();
}
}
void start_ticking()
{
ticking = true;
using namespace std::chrono_literals;
timeout_elapsed = 0ms;
RAFT_INFO_FMT("Election timer has become active");
}
void reset_last_ack_timeouts()
{
for (auto& node : all_other_nodes)
{
using namespace std::chrono_literals;
node.second.last_ack_timeout = 0ms;
}
}
Configuration::Nodes get_latest_configuration_unsafe() const override
{
if (configurations.empty())
{
return {};
}
return configurations.back().nodes;
}
Configuration::Nodes get_latest_configuration() override
{
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
return get_latest_configuration_unsafe();
}
ccf::kv::ConsensusDetails get_details() override
{
ccf::kv::ConsensusDetails details;
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
details.primary_id = leader_id;
details.current_view = state->current_view;
details.ticking = ticking;
details.leadership_state = state->leadership_state;
details.membership_state = state->membership_state;
if (is_retired())
{
details.retirement_phase = state->retirement_phase;
}
for (auto const& conf : configurations)
{
details.configs.push_back(conf);
}
for (auto& [k, v] : all_other_nodes)
{
details.acks[k] = {
v.match_idx, static_cast<size_t>(v.last_ack_timeout.count())};
}
details.reconfiguration_type = ccf::ReconfigurationType::ONE_TRANSACTION;
return details;
}
bool replicate(const ccf::kv::BatchVector& entries, Term term) override
{
std::lock_guard<ccf::pal::Mutex> guard(state->lock);
if (state->leadership_state != ccf::kv::LeadershipState::Leader)
{
RAFT_DEBUG_FMT(
"Failed to replicate {} items: not leader", entries.size());
rollback(state->last_idx);
return false;
}
if (term != state->current_view)
{
RAFT_DEBUG_FMT(
"Failed to replicate {} items at term {}, current term is {}",
entries.size(),
term,
state->current_view);
return false;
}
if (is_retired_committed())
{
RAFT_DEBUG_FMT(
"Failed to replicate {} items: node retirement is complete",
entries.size());
rollback(state->last_idx);
return false;
}
RAFT_DEBUG_FMT("Replicating {} entries", entries.size());
for (const auto& [index, data, is_globally_committable, hooks] : entries)
{
bool globally_committable = is_globally_committable;
if (index != state->last_idx + 1)
{
return false;
}
RAFT_DEBUG_FMT(
"Replicated on leader {}: {}{} ({} hooks)",
state->node_id,
index,
(globally_committable ? " committable" : ""),
hooks->size());
#ifdef CCF_RAFT_TRACING
nlohmann::json j = {};
j["function"] = "replicate";
j["state"] = *state;
COMMITTABLE_INDICES(j["state"], state);
j["view"] = term;
j["seqno"] = index;
j["globally_committable"] = globally_committable;
RAFT_TRACE_JSON_OUT(j);
#endif
for (auto& hook : *hooks)
{
hook->call(this);
}
if (globally_committable)
{
RAFT_DEBUG_FMT(
"membership: {} leadership: {}",
state->membership_state,
state->leadership_state);
if (
state->membership_state == ccf::kv::MembershipState::Retired &&
state->retirement_phase == ccf::kv::RetirementPhase::Ordered)
{
become_retired(index, ccf::kv::RetirementPhase::Signed);
}
state->committable_indices.push_back(index);
start_ticking_if_necessary();
// Reset should_sign here - whenever we see a committable entry we
// don't need to produce _another_ signature
should_sign = false;
}
state->last_idx = index;
ledger->put_entry(
*data, globally_committable, state->current_view, index);
entry_size_not_limited += data->size();
entry_count++;
state->view_history.update(index, state->current_view);
if (entry_size_not_limited >= append_entries_size_limit)
{
update_batch_size();
entry_count = 0;
entry_size_not_limited = 0;
for (const auto& it : all_other_nodes)
{
RAFT_DEBUG_FMT("Sending updates to follower {}", it.first);
send_append_entries(it.first, it.second.sent_idx + 1);
}
}
}
// Try to advance commit at once if there are no other nodes.
if (other_nodes_in_active_configs().size() == 0)
{
update_commit();
}
return true;
}
void recv_message(
const ccf::NodeId& from, const uint8_t* data, size_t size) override
{
auto type = serialized::peek<RaftMsgType>(data, size);
try
{
switch (type)
{
case raft_append_entries:
{
AppendEntries r =
channels->template recv_authenticated<AppendEntries>(
from, data, size);
recv_append_entries(from, r, data, size);
break;
}
case raft_append_entries_response:
{
AppendEntriesResponse r =
channels->template recv_authenticated<AppendEntriesResponse>(
from, data, size);
recv_append_entries_response(from, r);
break;
}
case raft_request_pre_vote:
{
RequestPreVote r =
channels->template recv_authenticated<RequestPreVote>(
from, data, size);
recv_request_pre_vote(from, r);
break;
}
case raft_request_vote:
{
RequestVote r = channels->template recv_authenticated<RequestVote>(
from, data, size);
recv_request_vote(from, r);
break;
}
case raft_request_pre_vote_response:
{
RequestPreVoteResponse r =
channels->template recv_authenticated<RequestPreVoteResponse>(
from, data, size);
recv_request_pre_vote_response(from, r);
break;
}
case raft_request_vote_response:
{
RequestVoteResponse r =
channels->template recv_authenticated<RequestVoteResponse>(
from, data, size);
recv_request_vote_response(from, r);
break;
}
case raft_propose_request_vote:
{
ProposeRequestVote r =
channels->template recv_authenticated<ProposeRequestVote>(
from, data, size);
recv_propose_request_vote(from, r);
break;
}
default:
{
RAFT_FAIL_FMT("Unhandled AFT message type: {}", type);
}
}
}
catch (const ccf::NodeToNode::DroppedMessageException& e)
{
RAFT_INFO_FMT("Dropped invalid message from {}", e.from);
return;
}
catch (const serialized::InsufficientSpaceException& ise)
{
RAFT_FAIL_FMT("Failed to parse message: {}", ise.what());
return;
}
catch (const std::exception& e)
{
LOG_FAIL_FMT("Exception in {}", __PRETTY_FUNCTION__);
LOG_DEBUG_FMT("Error: {}", e.what());
return;
}
}
void periodic(std::chrono::milliseconds elapsed) override
{
std::unique_lock<ccf::pal::Mutex> guard(state->lock);
timeout_elapsed += elapsed;
if (state->leadership_state == ccf::kv::LeadershipState::Leader)
{
if (timeout_elapsed >= request_timeout)
{
using namespace std::chrono_literals;
timeout_elapsed = 0ms;
update_batch_size();
// Send newly available entries to all other nodes.
for (const auto& node : all_other_nodes)
{
send_append_entries(node.first, node.second.sent_idx + 1);
}
}
for (auto& node : all_other_nodes)
{
node.second.last_ack_timeout += elapsed;
}
bool every_active_config_has_a_quorum = std::all_of(
configurations.begin(),
configurations.end(),
[this](const Configuration& conf) {
size_t live_nodes_in_config = 0;
for (auto const& node : conf.nodes)
{
auto search = all_other_nodes.find(node.first);
if (
// if a (non-self) node is in a configuration, then it is in
// all_other_nodes. So if a node in a configuration is not found
// in all_other_nodes, it must be self, and hence is live
search == all_other_nodes.end() ||
// Otherwise we use the most recent ack as a failure probe
search->second.last_ack_timeout < election_timeout)
{
++live_nodes_in_config;
}
else
{
RAFT_DEBUG_FMT(
"No ack received from {} in last {}",
node.first,
election_timeout);
}
}
return live_nodes_in_config >= get_quorum(conf.nodes.size());
});
if (!every_active_config_has_a_quorum)
{
// CheckQuorum: The primary automatically steps down if there are no
// active configuration in which it has heard back from a majority of
// backups within an election timeout.
// Also see CheckQuorum action in tla/ccfraft.tla.
RAFT_INFO_FMT(
"Stepping down as leader {}: No ack received from a majority of "
"backups in last {}",
state->node_id,
election_timeout);
become_follower();
}
}
else
{
if (
!is_retired_committed() && ticking &&
timeout_elapsed >= election_timeout)
{
// Start an election.
if (state->pre_vote_enabled)
{
become_pre_vote_candidate();
}
else
{
become_candidate();
}
}
}
}
private:
Index find_highest_possible_match(const ccf::TxID& tx_id)
{
// Find the highest TxID this node thinks exists, which is still
// compatible with the given tx_id. That is, given T.n, find largest n'
// such that n' <= n && term_of(n') == T' && T' <= T. This may be T.n
// itself, if this node holds that index. Otherwise, examine the final
// entry in each term, counting backwards, until we find one which is
// still possible.
Index probe_index = std::min(tx_id.seqno, state->last_idx);
Term term_of_probe = state->view_history.view_at(probe_index);
while (term_of_probe > tx_id.view)
{
// Next possible match is the end of the previous term, which is
// 1-before the start of the currently considered term. Anything after
// that must have a term which is still too high.
probe_index = state->view_history.start_of_view(term_of_probe);
if (probe_index > 0)
{
--probe_index;
}
term_of_probe = state->view_history.view_at(probe_index);
}
RAFT_TRACE_FMT(
"Looking for match with {}.{}, from {}.{}, best answer is {}",
tx_id.view,
tx_id.seqno,
state->view_history.view_at(state->last_idx),
state->last_idx,
probe_index);
return probe_index;