-
Notifications
You must be signed in to change notification settings - Fork 97
/
Copy pathra_server_proc.erl
2125 lines (1980 loc) · 91.6 KB
/
ra_server_proc.erl
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
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2017-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries.
%%
%% @hidden
-module(ra_server_proc).
-behaviour(gen_statem).
-compile({inline, [handle_raft_state/3]}).
-include("ra.hrl").
-include("ra_server.hrl").
%% State functions
-export([
post_init/3,
recover/3,
recovered/3,
leader/3,
pre_vote/3,
candidate/3,
follower/3,
receive_snapshot/3,
await_condition/3,
terminating_leader/3,
terminating_follower/3
]).
%% gen_statem callbacks
-export([
init/1,
format_status/1,
handle_event/4,
terminate/3,
code_change/4,
callback_mode/0
]).
%% API
-export([start_link/1,
command/3,
cast_command/2,
cast_command/3,
query/5,
state_query/3,
local_state_query/3,
trigger_election/2,
ping/2,
log_fold/4,
transfer_leadership/3,
force_shrink_members_to_current_member/1
]).
-export([send_rpc/3]).
-ifdef(TEST).
-export([leader_call/3,
local_call/3]).
-endif.
-define(DEFAULT_BROADCAST_TIME, 100).
-define(DEFAULT_ELECTION_MULT, 5).
-define(TICK_INTERVAL_MS, 1000).
-define(DEFAULT_AWAIT_CONDITION_TIMEOUT, 30000).
%% Utilisation average calculations are all in μs.
-define(INSTALL_SNAP_RPC_TIMEOUT, 120 * 1000).
-define(HANDLE_EFFECTS(Effects, EvtType, State0),
handle_effects(?FUNCTION_NAME, Effects, EvtType, State0)).
-define(ASYNC_DIST(Node, Send),
case Node == node() of
true ->
Send,
ok;
false ->
%% use async_dist for remote sends
process_flag(async_dist, true),
Send,
process_flag(async_dist, false),
ok
end).
-type query_fun() :: ra:query_fun().
-type query_options() :: #{condition => ra:query_condition()}.
-type ra_command() :: {ra_server:command_type(), term(),
ra_server:command_reply_mode()}.
-type ra_leader_call_ret(Result) :: {ok, Result, Leader::ra_server_id()} |
{error, term()} |
{timeout, ra_server_id()}.
-type ra_local_call_ret(Result) :: {ok, Result, LocalServer::ra_server_id()} |
{error, term()} |
{timeout, ra_server_id()}.
-type ra_cmd_ret() :: ra_leader_call_ret(term()).
-type gen_statem_start_ret() :: {ok, pid()} | ignore | {error, term()}.
%% ra_event types
-type ra_event_reject_detail() :: {not_leader, Leader :: option(ra_server_id()),
ra_server:command_correlation()}.
-type ra_event_body() ::
% used for notifying senders of the ultimate fate of their command
% sent using ra:pipeline_command/3|4
{applied, [{ra_server:command_correlation(), Reply :: term()}]} |
{rejected, ra_event_reject_detail()} |
% used to send message side-effects emitted by the state machine
{machine, term()}.
-type ra_event() :: {ra_event, Sender :: ra_server_id(), ra_event_body()}.
%% the Sender is the ra process that emitted the ra_event.
-type server_loc() :: ra_server_id() | [ra_server_id()].
-export_type([ra_leader_call_ret/1,
ra_local_call_ret/1,
ra_cmd_ret/0,
safe_call_ret/1,
ra_event_reject_detail/0,
ra_event/0,
ra_event_body/0,
query_options/0]).
%% the ra server proc keeps monitors on behalf of different components
%% the state machine, log and server code. The tag is used to determine
%% which component to dispatch the down to
-record(conf, {log_id :: unicode:chardata(),
name :: atom(),
cluster_name :: term(),
broadcast_time = ?DEFAULT_BROADCAST_TIME :: non_neg_integer(),
tick_timeout :: non_neg_integer(),
await_condition_timeout :: non_neg_integer(),
ra_event_formatter :: undefined | {module(), atom(), [term()]},
flush_commands_size = ?FLUSH_COMMANDS_SIZE :: non_neg_integer(),
snapshot_chunk_size = ?DEFAULT_SNAPSHOT_CHUNK_SIZE :: non_neg_integer(),
receive_snapshot_timeout = ?DEFAULT_RECEIVE_SNAPSHOT_TIMEOUT :: non_neg_integer(),
install_snap_rpc_timeout :: non_neg_integer(),
aten_poll_interval = 1000 :: non_neg_integer(),
counter :: undefined | counters:counters_ref()
}).
-record(state, {conf :: #conf{},
server_state :: ra_server:ra_server_state(),
monitors = ra_monitors:init() :: ra_monitors:state(),
pending_commands = [] :: [{{pid(), any()}, term()}],
leader_monitor :: reference() | undefined,
leader_last_seen :: integer() | undefined,
low_priority_commands :: ra_ets_queue:state(),
election_timeout_set = false :: boolean(),
%% the log index last time gc was forced
pending_notifys = #{} :: #{pid() => [term()]},
pending_queries = [] :: [{ra:query_condition(),
gen_statem:from(),
query_fun()}]
}).
%%%===================================================================
%%% API
%%%===================================================================
-spec start_link(ra_server:ra_server_config()) -> gen_statem_start_ret().
start_link(Config = #{id := Id}) ->
Name = ra_lib:ra_server_id_to_local_name(Id),
gen_statem:start_link({local, Name}, ?MODULE, Config, []).
-spec command(server_loc(), ra_command(), timeout()) ->
ra_cmd_ret().
command(ServerLoc, Cmd, Timeout) ->
leader_call(ServerLoc, {command, normal, Cmd}, Timeout).
-spec cast_command(ra_server_id(), ra_command()) -> ok.
cast_command(ServerId, Cmd) ->
gen_statem:cast(ServerId, {command, low, Cmd}).
-spec cast_command(ra_server_id(), ra_server:command_priority(), ra_command()) -> ok.
cast_command(ServerId, Priority, Cmd) ->
gen_statem:cast(ServerId, {command, Priority, Cmd}).
-spec query(server_loc(), query_fun(),
local | consistent | leader,
query_options(),
timeout()) ->
ra_server_proc:ra_leader_call_ret({ra_idxterm(), Reply :: term()})
| ra_server_proc:ra_leader_call_ret(Reply :: term())
| {ok, {ra_idxterm(), Reply :: term()}, not_known}.
query(ServerLoc, QueryFun, local, Options, Timeout)
when map_size(Options) =:= 0 ->
statem_call(ServerLoc, {local_query, QueryFun}, Timeout);
query(ServerLoc, QueryFun, local, Options, Timeout) ->
statem_call(ServerLoc, {local_query, QueryFun, Options}, Timeout);
query(ServerLoc, QueryFun, leader, Options, Timeout)
when map_size(Options) =:= 0 ->
leader_call(ServerLoc, {local_query, QueryFun}, Timeout);
query(ServerLoc, QueryFun, leader, Options, Timeout) ->
leader_call(ServerLoc, {local_query, QueryFun, Options}, Timeout);
query(ServerLoc, QueryFun, consistent, _Options, Timeout) ->
leader_call(ServerLoc, {consistent_query, QueryFun}, Timeout).
-spec log_fold(ra_server_id(), fun(), term(), integer()) -> term().
log_fold(ServerId, Fun, InitialState, Timeout) ->
gen_statem:call(ServerId, {log_fold, Fun, InitialState}, Timeout).
%% used to query the raft state rather than the machine state
-spec state_query(server_loc(),
all |
overview |
voters |
members |
members_info |
initial_members |
machine, timeout()) ->
ra_leader_call_ret(term()).
state_query(ServerLoc, Spec, Timeout) ->
leader_call(ServerLoc, {state_query, Spec}, Timeout).
-spec local_state_query(server_loc(),
all |
overview |
voters |
members |
members_info |
initial_members |
machine, timeout()) ->
ra_local_call_ret(term()).
local_state_query(ServerLoc, Spec, Timeout) ->
local_call(ServerLoc, {state_query, Spec}, Timeout).
-spec trigger_election(ra_server_id(), timeout()) -> ok.
trigger_election(ServerId, Timeout) ->
gen_statem:call(ServerId, trigger_election, Timeout).
-spec transfer_leadership(ra_server_id(), ra_server_id(), timeout()) ->
ok | already_leader | {error, term()} | {timeout, ra_server_id()}.
transfer_leadership(ServerId, TargetServerId, Timeout) ->
leader_call(ServerId, {transfer_leadership, TargetServerId}, Timeout).
-spec force_shrink_members_to_current_member(ra_server_id()) -> ok.
force_shrink_members_to_current_member(ServerId) ->
gen_statem_safe_call(ServerId, force_member_change, 5000).
-spec ping(ra_server_id(), timeout()) -> safe_call_ret({pong, states()}).
ping(ServerId, Timeout) ->
gen_statem_safe_call(ServerId, ping, Timeout).
leader_call(ServerLoc, Msg, Timeout) ->
statem_call(ServerLoc, {leader_call, Msg}, Timeout).
local_call(ServerLoc, Msg, Timeout) ->
statem_call(ServerLoc, {local_call, Msg}, Timeout).
statem_call(ServerIds, Msg, Timeout)
when is_list(ServerIds) ->
multi_statem_call(ServerIds, Msg, [], Timeout);
statem_call(ServerId, Msg, Timeout) ->
case gen_statem_safe_call(ServerId, Msg, Timeout) of
{redirect, Leader} ->
statem_call(Leader, Msg, Timeout);
{wrap_reply, Reply} ->
{ok, Reply, ServerId};
{error, _} = E ->
E;
timeout ->
{timeout, ServerId};
Reply ->
Reply
end.
multi_statem_call([ServerId | ServerIds], Msg, Errs, Timeout) ->
case statem_call(ServerId, Msg, Timeout) of
{Tag, Info} = E
when Tag == timeout orelse
(Tag == error andalso
(Info == noproc orelse
Info == nodedown orelse
Info == shutdown orelse
Info == system_not_started)) ->
%% these are the retryable errors, any others we consider
%% genuine errors that a retry will not fix
case ServerIds of
[] ->
{error, {no_more_servers_to_try, [E | Errs]}};
_ ->
multi_statem_call(ServerIds, Msg, [E | Errs], Timeout)
end;
Reply ->
Reply
end.
%%%===================================================================
%%% gen_statem callbacks
%%%===================================================================
init(#{reply_to := ReplyTo} = Config) ->
%% we have a reply to key, perform init async
{ok, post_init, maps:remove(reply_to, Config),
[{next_event, internal, {go, ReplyTo}}]};
init(Config) ->
%% no reply_to key, must have been started by an older node run synchronous
%% init
State = do_init(Config),
{ok, recover, State, [{next_event, cast, go}]}.
do_init(#{id := Id,
cluster_name := ClusterName} = Config0) ->
Key = ra_lib:ra_server_id_to_local_name(Id),
true = ets:insert(ra_state, {Key, init, unknown}),
process_flag(trap_exit, true),
Config = #{counter := Counter,
system_config := #{names := Names} = SysConf} = maps:merge(config_defaults(Id),
Config0),
MsgQData = maps:get(message_queue_data, SysConf, off_heap),
MinBinVheapSize = maps:get(server_min_bin_vheap_size, SysConf,
?MIN_BIN_VHEAP_SIZE),
MinHeapSize = maps:get(server_min_heap_size, SysConf, ?MIN_BIN_VHEAP_SIZE),
process_flag(message_queue_data, MsgQData),
process_flag(min_bin_vheap_size, MinBinVheapSize),
process_flag(min_heap_size, MinHeapSize),
%% wait for wal for a bit before initialising the server state and log
#{cluster := Cluster} = ServerState = ra_server:init(Config),
LogId = ra_server:log_id(ServerState),
UId = ra_server:uid(ServerState),
% ensure ra_directory has the new pid
#{names := Names} = SysConf,
ok = ra_directory:register_name(Names, UId, self(),
maps:get(parent, Config, undefined), Key,
ClusterName),
% ensure each relevant erlang node is connected
PeerNodes = [PeerNode ||
{_, PeerNode} <- maps:keys(maps:remove(Id, Cluster))],
case PeerNodes -- nodes() of
[] ->
%% all peer nodes are connected
ok;
DisconnectedNodes ->
%% as most messages are sent using noconnect we explicitly attempt to
%% connect to all relevant nodes
_ = spawn(fun () ->
[net_kernel:connect_node(N)
|| N <- DisconnectedNodes]
end),
ok
end,
TickTime = maps:get(tick_timeout, Config),
InstallSnapRpcTimeout = maps:get(install_snap_rpc_timeout, Config),
AwaitCondTimeout = maps:get(await_condition_timeout, Config),
RaEventFormatterMFA = maps:get(ra_event_formatter, Config, undefined),
FlushCommandsSize = maps:get(low_priority_commands_flush_size, SysConf,
?FLUSH_COMMANDS_SIZE),
SnapshotChunkSize = maps:get(snapshot_chunk_size, SysConf,
?DEFAULT_SNAPSHOT_CHUNK_SIZE),
ReceiveSnapshotTimeout = maps:get(receive_snapshot_timeout, SysConf,
?DEFAULT_RECEIVE_SNAPSHOT_TIMEOUT),
AtenPollInt = application:get_env(aten, poll_interval, 1000),
State = #state{conf = #conf{log_id = LogId,
cluster_name = ClusterName,
name = Key,
tick_timeout = TickTime,
await_condition_timeout = AwaitCondTimeout,
ra_event_formatter = RaEventFormatterMFA,
flush_commands_size = FlushCommandsSize,
snapshot_chunk_size = SnapshotChunkSize,
install_snap_rpc_timeout = InstallSnapRpcTimeout,
receive_snapshot_timeout = ReceiveSnapshotTimeout,
aten_poll_interval = AtenPollInt,
counter = Counter},
low_priority_commands = ra_ets_queue:new(),
server_state = ServerState},
ok = net_kernel:monitor_nodes(true, [nodedown_reason]),
State.
%% callback mode
callback_mode() -> [state_functions, state_enter].
%%%===================================================================
%%% State functions
%%%===================================================================
post_init(enter, _OldState, State) ->
{keep_state, State, []};
post_init(internal, {go, {ReplyToRef, ReplyToPid}}, Config) ->
State = do_init(Config),
ReplyToPid ! {ReplyToRef, ok},
{next_state, recover, State, [{next_event, internal, go}]}.
recover(enter, OldState, State0) ->
{State, Actions} = handle_enter(?FUNCTION_NAME, OldState, State0),
{keep_state, State, Actions};
recover(internal, go, State = #state{server_state = ServerState0}) ->
ServerState = ra_server:recover(ServerState0),
incr_counter(State#state.conf, ?C_RA_SRV_GCS, 1),
%% we have to issue the next_event here so that the recovered state is
%% only passed through very briefly
next_state(recovered, State#state{server_state = ServerState},
[{next_event, internal, next}]);
recover(_, _, State) ->
% all other events need to be postponed until we can return
% `next_event` from init
{keep_state, State, {postpone, true}}.
%% this is a passthrough state to allow state machines to emit node local
%% effects post recovery
recovered(enter, OldState, State0) ->
{State, Actions} = handle_enter(?FUNCTION_NAME, OldState, State0),
ok = record_cluster_change(State),
{keep_state, State, Actions};
recovered(internal, next, #state{server_state = ServerState} = State) ->
true = erlang:garbage_collect(),
_ = ets:insert(ra_metrics, ra_server:metrics(ServerState)),
next_state(follower, State, set_tick_timer(State, [])).
leader(enter, OldState, #state{low_priority_commands = Delayed0} = State0) ->
{State, Actions} = handle_enter(?FUNCTION_NAME, OldState, State0),
Delayed = case OldState of
await_condition ->
%% if we're returning from await_condition we may still
%% have valid delayed commands to schedule
schedule_command_flush(Delayed0),
Delayed0;
_ ->
%% for any other state it is best to just reset the
%% delayed commands
ra_ets_queue:reset(Delayed0)
end,
ok = record_cluster_change(State),
{keep_state, State#state{leader_last_seen = undefined,
pending_notifys = #{},
low_priority_commands = Delayed,
election_timeout_set = false}, Actions};
leader(EventType, {leader_call, Msg}, State) ->
% no need to redirect
leader(EventType, Msg, State);
leader(EventType, {local_call, Msg}, State) ->
leader(EventType, Msg, State);
leader(EventType, {leader_cast, Msg}, State) ->
leader(EventType, Msg, State);
leader(EventType, {command, normal, {CmdType, Data, ReplyMode}},
#state{conf = Conf} = State0) ->
case validate_reply_mode(ReplyMode) of
ok ->
%% normal priority commands are written immediately
Cmd = make_command(CmdType, EventType, Data, ReplyMode),
{NextState, State1, Effects} = handle_leader({command, Cmd}, State0),
{State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
case NextState of
leader ->
{keep_state, State, Actions};
_ ->
next_state(NextState, State, Actions)
end;
Error ->
ok = incr_counter(Conf, ?C_RA_SRV_INVALID_REPLY_MODE_COMMANDS, 1),
case EventType of
{call, From} ->
{keep_state, State0, [{reply, From, Error}]};
_ ->
{keep_state, State0, []}
end
end;
leader(EventType, {command, low, {'$usr', Data, ReplyMode}},
#state{conf = Conf,
low_priority_commands = Delayed} = State0) ->
%% only user commands can be low priority
case validate_reply_mode(ReplyMode) of
ok ->
%% cache the low priority command until the flush_commands message
%% arrives
Cmd = make_command('$usr', EventType, Data, ReplyMode),
%% if there are no prior delayed commands
%% (and thus no action queued to do so)
%% queue a state timeout to flush them
%% We use a cast to ourselves instead of a zero timeout as we want
%% to get onto the back of the erlang mailbox not just the current
%% gen_statem event buffer.
case ra_ets_queue:len(Delayed) of
0 ->
ok = gen_statem:cast(self(), flush_commands);
_ ->
ok
end,
State = State0#state{low_priority_commands =
ra_ets_queue:in(Cmd, Delayed)},
{keep_state, State, []};
Error ->
ok = incr_counter(Conf, ?C_RA_SRV_INVALID_REPLY_MODE_COMMANDS, 1),
case EventType of
{call, From} ->
{keep_state, State0, [{reply, From, Error}]};
_ ->
{keep_state, State0, []}
end
end;
leader(EventType, {command, low, Cmd}, #state{} = State) ->
%% non user low priority commands are upgraded to normal priority
leader(EventType, {command, normal, Cmd}, State);
leader(EventType, {aux_command, Cmd}, State0) ->
{_, ServerState, Effects} = ra_server:handle_aux(?FUNCTION_NAME, EventType,
Cmd, State0#state.server_state),
{State, Actions} =
?HANDLE_EFFECTS(Effects, EventType,
State0#state{server_state = ServerState}),
{keep_state, State#state{server_state = ServerState}, Actions};
leader(EventType, flush_commands,
#state{conf = #conf{flush_commands_size = Size},
low_priority_commands = Delayed0} = State0) ->
{Commands, Delayed} = ra_ets_queue:take(Size, Delayed0),
%% write a batch of delayed commands
{NextState, State1, Effects} = handle_leader({commands, Commands}, State0),
State2 = State1#state{low_priority_commands = Delayed},
{State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State2),
case NextState of
leader ->
schedule_command_flush(Delayed),
{keep_state, State#state{low_priority_commands = Delayed}, Actions};
_ ->
next_state(NextState, State, Actions)
end;
leader({call, _From} = EventType, {local_query, QueryFun}, State) ->
leader(EventType, {local_query, QueryFun, #{}}, State);
leader({call, From}, {local_query, QueryFun, Options}, State) ->
perform_or_delay_local_query(leader, From, QueryFun, Options, State);
leader({call, From}, {state_query, Spec}, State) ->
Reply = {ok, do_state_query(Spec, State), id(State)},
{keep_state, State, [{reply, From, Reply}]};
leader({call, From}, {consistent_query, QueryFun},
#state{conf = Conf,
server_state = ServerState0} = State0) ->
{leader, ServerState1, Effects} =
ra_server:handle_leader({consistent_query, From, QueryFun},
ServerState0),
incr_counter(Conf, ?C_RA_SRV_CONSISTENT_QUERIES, 1),
{State1, Actions} =
?HANDLE_EFFECTS(Effects, {call, From},
State0#state{server_state = ServerState1}),
{keep_state, State1, Actions};
leader({call, From}, ping, State) ->
{keep_state, State, [{reply, From, {pong, leader}}]};
leader(info, {node_event, _Node, _Evt}, State) ->
{keep_state, State, []};
leader(info, {'DOWN', _MRef, process, Pid, Info}, State0) ->
handle_process_down(Pid, Info, ?FUNCTION_NAME, State0);
leader(info, {Status, Node, InfoList}, State0)
when Status =:= nodedown orelse
Status =:= nodeup ->
handle_node_status_change(Node, Status, InfoList, ?FUNCTION_NAME, State0);
leader(info, {update_peer, PeerId, Update}, State0) ->
State = update_peer(PeerId, Update, State0),
{keep_state, State, []};
leader(_, tick_timeout, State0) ->
{State1, RpcEffs} = make_rpcs(State0),
ServerState0 = State1#state.server_state,
Effects = ra_server:tick(ServerState0),
ServerState = ra_server:log_tick(ServerState0),
{State2, Actions} = ?HANDLE_EFFECTS(RpcEffs ++ Effects ++ [{aux, tick}],
cast, State1#state{server_state = ServerState}),
%% try sending any pending applied notifications again
State = send_applied_notifications(State2, #{}),
{keep_state, handle_tick_metrics(State),
set_tick_timer(State, Actions)};
leader({timeout, Name}, machine_timeout, State0) ->
% the machine timer timed out, add a timeout message
Cmd = make_command('$usr', cast, {timeout, Name}, noreply),
{leader, State1, Effects} = handle_leader({command, Cmd}, State0),
{State, Actions} = ?HANDLE_EFFECTS(Effects, cast, State1),
{keep_state, State, Actions};
leader({call, From}, trigger_election, State) ->
{keep_state, State, [{reply, From, ok}]};
leader({call, From}, {log_fold, Fun, Term}, State) ->
fold_log(From, Fun, Term, State);
leader(EventType, Msg, State0) ->
case handle_leader(Msg, State0) of
{leader, State1, Effects1} ->
{State, Actions} = ?HANDLE_EFFECTS(Effects1, EventType, State1),
{keep_state, State, Actions};
{stop, State1, Effects} ->
% interact before shutting down in case followers need
% to know about the new commit index
{State, _Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
{stop, normal, State};
{delete_and_terminate, State1, Effects} ->
{State2, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
State = send_rpcs(State2),
case ra_server:is_fully_replicated(State#state.server_state) of
true ->
{stop, {shutdown, delete}, State};
false ->
next_state(terminating_leader, State, Actions)
end;
{await_condition, State1, Effects1} ->
{State, Actions} = ?HANDLE_EFFECTS(Effects1, EventType, State1),
?DEBUG_IF(is_command(Msg), "~ts: postponing ~0P",
[log_id(State0), Msg, 10]),
next_state(await_condition, State,
[{postpone, is_command(Msg)} | Actions]);
{NextState, State1, Effects1} ->
{State, Actions} = ?HANDLE_EFFECTS(Effects1, EventType, State1),
next_state(NextState, State, Actions)
end.
candidate(enter, OldState, State0) ->
{State1, Actions0} = handle_enter(?FUNCTION_NAME, OldState, State0),
{State, Actions} = maybe_set_election_timeout(short, State1, Actions0),
{keep_state, State, Actions};
candidate({call, From}, {leader_call, Msg},
#state{pending_commands = Pending} = State) ->
{keep_state, State#state{pending_commands = [{From, Msg} | Pending]}};
candidate(EventType, {local_call, Msg}, State) ->
candidate(EventType, Msg, State);
candidate(cast, {command, _Priority,
{_CmdType, _Data, {notify, Corr, Pid}}},
State) ->
_ = reject_command(Pid, Corr, State),
{keep_state, State, []};
candidate({call, _From} = EventType, {local_query, QueryFun}, State) ->
candidate(EventType, {local_query, QueryFun, #{}}, State);
candidate({call, From}, {local_query, QueryFun, Options}, State) ->
perform_or_delay_local_query(candidate, From, QueryFun, Options, State);
candidate({call, From}, {state_query, Spec}, State) ->
Reply = {ok, do_state_query(Spec, State), id(State)},
{keep_state, State, [{reply, From, Reply}]};
candidate({call, From}, ping, State) ->
{keep_state, State, [{reply, From, {pong, candidate}}]};
candidate(info, {node_event, _Node, _Evt}, State) ->
{keep_state, State};
candidate(_, tick_timeout, State0) ->
State = maybe_persist_last_applied(State0),
{keep_state, handle_tick_metrics(State), set_tick_timer(State, [])};
candidate({call, From}, trigger_election, State) ->
{keep_state, State, [{reply, From, ok}]};
candidate(EventType, Msg, State0) ->
case handle_candidate(Msg, State0) of
{candidate, State1, Effects} ->
{State2, Actions0} = ?HANDLE_EFFECTS(Effects, EventType, State1),
{State, Actions} = maybe_set_election_timeout(medium, State2, Actions0),
{keep_state, State, Actions};
{follower, State1, Effects} ->
{State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
next_state(follower, State, Actions);
{leader, State1, Effects} ->
{State, Actions0} = ?HANDLE_EFFECTS(Effects, EventType, State1),
%% reset the tick timer to avoid it triggering early after a leader
%% change
Actions = set_tick_timer(State, Actions0),
next_state(leader, State, Actions)
end.
pre_vote(enter, OldState, #state{leader_monitor = MRef} = State0) ->
_ = stop_monitor(MRef),
{State1, Actions0} = handle_enter(?FUNCTION_NAME, OldState, State0),
{State, Actions} = maybe_set_election_timeout(long, State1, Actions0),
{keep_state, State#state{leader_monitor = undefined}, Actions};
pre_vote({call, From}, {leader_call, Msg},
State = #state{pending_commands = Pending}) ->
{keep_state, State#state{pending_commands = [{From, Msg} | Pending]}};
pre_vote(EventType, {local_call, Msg}, State) ->
pre_vote(EventType, Msg, State);
pre_vote(cast, {command, _Priority,
{_CmdType, _Data, {notify, Corr, Pid}}},
State) ->
_ = reject_command(Pid, Corr, State),
{keep_state, State, []};
pre_vote({call, _From} = EventType, {local_query, QueryFun}, State) ->
pre_vote(EventType, {local_query, QueryFun, #{}}, State);
pre_vote({call, From}, {local_query, QueryFun, Options}, State) ->
perform_or_delay_local_query(pre_vote, From, QueryFun, Options, State);
pre_vote({call, From}, {state_query, Spec}, State) ->
Reply = {ok, do_state_query(Spec, State), id(State)},
{keep_state, State, [{reply, From, Reply}]};
pre_vote({call, From}, ping, State) ->
{keep_state, State, [{reply, From, {pong, pre_vote}}]};
pre_vote(info, {node_event, _Node, _Evt}, State) ->
{keep_state, State};
pre_vote(info, {Status, Node, InfoList}, State0)
when Status =:= nodedown orelse Status =:= nodeup ->
handle_node_status_change(Node, Status, InfoList, ?FUNCTION_NAME, State0);
pre_vote(info, {'DOWN', _MRef, process, Pid, Info}, State0) ->
handle_process_down(Pid, Info, ?FUNCTION_NAME, State0);
pre_vote(_, tick_timeout, State0) ->
State = maybe_persist_last_applied(State0),
{keep_state, handle_tick_metrics(State), set_tick_timer(State, [])};
pre_vote({call, From}, trigger_election, State) ->
{keep_state, State, [{reply, From, ok}]};
pre_vote(EventType, Msg, State0) ->
case handle_pre_vote(Msg, State0) of
{pre_vote, State1, Effects} ->
{State2, Actions0} = ?HANDLE_EFFECTS(Effects, EventType, State1),
{State, Actions} = maybe_set_election_timeout(long, State2, Actions0),
{keep_state, State, Actions};
{follower, State1, Effects} ->
{State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
next_state(follower, State, Actions);
{candidate, State1, Effects} ->
{State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
next_state(candidate, State, Actions)
end.
follower(enter, OldState, #state{low_priority_commands = Delayed,
server_state = ServerState} = State0) ->
%% New cluster starts should be coordinated and elections triggered
%% explicitly hence if this is a new one we wait here.
%% Else we set an election timer
%% Set the timeout length based on the previous state
TimeoutLen = case OldState of
recovered -> short;
_ -> long
end,
{State1, Actions0} = handle_enter(?FUNCTION_NAME, OldState, State0),
{State, Actions} = case ra_server:is_new(ServerState) of
true ->
{State1, Actions0};
false ->
?DEBUG("~ts: is not new, setting "
"election timeout.",
[log_id(State0)]),
maybe_set_election_timeout(TimeoutLen, State1,
Actions0)
end,
Monitors = ra_monitors:remove_all(machine, State#state.monitors),
{keep_state, State#state{low_priority_commands = ra_ets_queue:reset(Delayed),
monitors = Monitors}, Actions};
follower({call, From}, {leader_call, Msg}, State) ->
maybe_redirect(From, Msg, State);
follower(EventType, {local_call, Msg}, State) ->
follower(EventType, Msg, State);
follower(_, {command, Priority, {_CmdType, Data, noreply}},
State) ->
% forward to leader
case leader_id(State) of
undefined ->
?WARN("~ts: leader cast - leader not known. "
"Command is dropped.", [log_id(State)]),
{keep_state, State, []};
LeaderId ->
?DEBUG("~ts: follower leader cast - redirecting to ~w ",
[log_id(State), LeaderId]),
ok = ra:pipeline_command(LeaderId, Data, no_correlation, Priority),
{keep_state, State, []}
end;
follower(cast, {command, _Priority,
{_CmdType, _Data, {notify, Corr, Pid}}},
State) ->
_ = reject_command(Pid, Corr, State),
{keep_state, State, []};
follower({call, _From} = EventType, {local_query, QueryFun}, State) ->
follower(EventType, {local_query, QueryFun, #{}}, State);
follower({call, From}, {local_query, QueryFun, Options}, State) ->
perform_or_delay_local_query(follower, From, QueryFun, Options, State);
follower({call, From}, {state_query, Spec}, State) ->
Reply = {ok, do_state_query(Spec, State), id(State)},
{keep_state, State, [{reply, From, Reply}]};
follower(EventType, {aux_command, Cmd}, State0) ->
{_, ServerState, Effects} = ra_server:handle_aux(?FUNCTION_NAME, EventType, Cmd,
State0#state.server_state),
{State, Actions} =
?HANDLE_EFFECTS(Effects, EventType,
State0#state{server_state = ServerState}),
{keep_state, State#state{server_state = ServerState}, Actions};
follower({call, From}, trigger_election, State) ->
?DEBUG("~ts: election triggered by ~w", [log_id(State), element(1, From)]),
{keep_state, State, [{reply, From, ok},
{next_event, cast, election_timeout}]};
follower({call, From}, ping, State) ->
{keep_state, State, [{reply, From, {pong, follower}}]};
follower(info, {'DOWN', MRef, process, _Pid, Info},
#state{leader_monitor = MRef} = State0) ->
?INFO("~ts: Leader monitor down with ~W, setting election timeout",
[log_id(State0), Info, 8]),
%% If the DOWN reason is something else than `noconnection', we know that
%% the leader process is really gone. We want to clear the leader ID we
%% know at this point, while a new election is running.
%%
%% This is to make sure that `follower_leader_change()' won't create a
%% useless monitor and more importantly, it won't redirect pending
%% commands to that old leader. This would cause the commands callers to
%% get a `noproc' error or a timeout.
State1 = case Info of
noconnection ->
State0;
_ ->
clear_leader_id(State0)
end,
TimeoutLen = case Info of
noconnection ->
short;
_ ->
%% if it isn't a network related error
%% set the shortest timeout
really_short
end,
{State, Actions} = maybe_set_election_timeout(TimeoutLen, State1, []),
{keep_state, State#state{leader_monitor = undefined}, Actions};
follower(info, {'DOWN', _MRef, process, Pid, Info}, State0) ->
handle_process_down(Pid, Info, ?FUNCTION_NAME, State0);
follower(info, {node_event, Node, down}, State0) ->
case leader_id(State0) of
{_, Node} ->
?DEBUG("~ts: Leader node ~w may be down, setting pre-vote timeout",
[log_id(State0), Node]),
{State, Actions} = maybe_set_election_timeout(long, State0, []),
{keep_state, State, Actions};
_ ->
{keep_state, State0, []}
end;
follower(info, {node_event, Node, up}, State) ->
case leader_id(State) of
{_, Node} when State#state.election_timeout_set ->
?DEBUG("~ts: Leader node ~w is back up, cancelling pre-vote timeout",
[log_id(State), Node]),
{keep_state,
State#state{election_timeout_set = false},
[{state_timeout, infinity, election_timeout}]};
_ ->
{keep_state, State, []}
end;
follower(info, {Status, Node, InfoList}, State0)
when Status =:= nodedown orelse Status =:= nodeup ->
handle_node_status_change(Node, Status, InfoList, ?FUNCTION_NAME, State0);
follower(_, tick_timeout, #state{server_state = ServerState0} = State0) ->
ServerState = ra_server:log_tick(ServerState0),
{State, Actions} = ?HANDLE_EFFECTS([{aux, tick}], cast,
State0#state{server_state = ServerState}),
{keep_state, handle_tick_metrics(State),
set_tick_timer(State, Actions)};
follower({call, From}, {log_fold, Fun, Term}, State) ->
fold_log(From, Fun, Term, State);
follower(EventType, Msg, #state{conf = #conf{name = Name},
server_state = SS0} = State0) ->
case handle_follower(Msg, State0) of
{follower, State1, Effects} ->
{State2, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
State = #state{server_state = SS} = follower_leader_change(State0, State2),
Membership0 = ra_server:get_membership(SS0),
case ra_server:get_membership(SS) of
Membership0 ->
ok;
Membership ->
true = ets:update_element(ra_state, Name, {3, Membership})
end,
{keep_state, State, Actions};
{pre_vote, State1, Effects} ->
{State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
next_state(pre_vote, State, Actions);
{await_condition, State1, Effects} ->
{State2, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
State = follower_leader_change(State0, State2),
next_state(await_condition, State, Actions);
{receive_snapshot, State1, Effects} ->
{State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
next_state(receive_snapshot, State, Actions);
{delete_and_terminate, State1, Effects} ->
{State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
next_state(terminating_follower, State, Actions)
end.
%% TODO: handle leader down abort snapshot and revert to follower
receive_snapshot(enter, OldState, State0 = #state{conf = Conf}) ->
#conf{receive_snapshot_timeout = ReceiveSnapshotTimeout} = Conf,
{State, Actions} = handle_enter(?FUNCTION_NAME, OldState, State0),
{keep_state, State,
[{state_timeout, ReceiveSnapshotTimeout, receive_snapshot_timeout}
| Actions]};
receive_snapshot(_, tick_timeout, State0) ->
{keep_state, State0, set_tick_timer(State0, [])};
receive_snapshot({call, From}, {leader_call, Msg}, State) ->
maybe_redirect(From, Msg, State);
receive_snapshot(EventType, {local_call, Msg}, State) ->
receive_snapshot(EventType, Msg, State);
receive_snapshot({call, _From} = EventType, {local_query, QueryFun}, State) ->
receive_snapshot(EventType, {local_query, QueryFun, #{}}, State);
receive_snapshot({call, From}, {local_query, QueryFun, Options}, State) ->
perform_or_delay_local_query(
receive_snapshot, From, QueryFun, Options, State);
receive_snapshot({call, From}, {state_query, Spec}, State) ->
Reply = {ok, do_state_query(Spec, State), id(State)},
{keep_state, State, [{reply, From, Reply}]};
receive_snapshot(EventType, Msg, State0) ->
case handle_receive_snapshot(Msg, State0) of
{receive_snapshot, State1, Effects} ->
{#state{conf = Conf} = State, Actions} =
?HANDLE_EFFECTS(Effects, EventType, State1),
TimeoutActions = case Msg of
#install_snapshot_rpc{} ->
%% Reset timeout only on receive snapshot progress.
[{state_timeout, Conf#conf.receive_snapshot_timeout,
receive_snapshot_timeout}];
_ ->
[]
end,
{keep_state, State, TimeoutActions ++ Actions};
{follower, State1, Effects} ->
{State2, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State1),
State = follower_leader_change(State0, State2),
next_state(follower, State, Actions)
end.
terminating_leader(enter, OldState, State0) ->
{State, Actions} = handle_enter(?FUNCTION_NAME, OldState, State0),
{keep_state, State, Actions};
terminating_leader(_EvtType, {command, _, _}, State0) ->
% do not process any further commands
{keep_state, State0, []};
terminating_leader(EvtType, Msg, State0) ->
LogName = log_id(State0),
?DEBUG("~ts: terminating leader received ~W", [LogName, Msg, 10]),
{State, Actions} = case leader(EvtType, Msg, State0) of
{next_state, terminating_leader, S, A} ->
{S, A};
{keep_state, S, A} ->
{S, A};
{stop, {shutdown, delete}, S} ->
{S, []}
end,
NS = State#state.server_state,
case ra_server:is_fully_replicated(NS) of
true ->
{stop, {shutdown, delete}, State};
false ->
?DEBUG("~ts: is not fully replicated after ~W",
[LogName, Msg, 7]),
{keep_state, send_rpcs(State), Actions}
end.
terminating_follower(enter, OldState, State0) ->
{State, Actions} = handle_enter(?FUNCTION_NAME, OldState, State0),
{keep_state, State, Actions};
terminating_follower(EvtType, Msg, State0) ->
% only process ra_log_events
LogName = log_id(State0),
?DEBUG("~ts: terminating follower received ~W", [LogName, Msg, 10]),
{State, Actions} = case follower(EvtType, Msg, State0) of
{next_state, terminating_follower, S, A} ->
{S, A};
{next_state, NextState, S, A} ->
?DEBUG("~ts: terminating follower requested state '~s'"
" - remaining in current state",
[LogName, NextState]),
{S, A};
{keep_state, S, A} ->
{S, A}
end,
case ra_server:is_fully_persisted(State#state.server_state) of
true ->
{stop, {shutdown, delete}, State};
false ->
?DEBUG("~ts: is not fully persisted after ~W",
[log_id(State), Msg, 7]),
{keep_state, State, Actions}
end.
await_condition(enter, OldState, #state{conf = Conf,
server_state = ServerState} = State0) ->
{State, Actions0} = handle_enter(?FUNCTION_NAME, OldState, State0),
Timeout = ra_server:get_condition_timeout(ServerState,
Conf#conf.await_condition_timeout),
Actions = [{state_timeout, Timeout, await_condition_timeout} | Actions0],
{keep_state, State, Actions};
await_condition({call, From}, {leader_call, Msg}, State) ->
maybe_redirect(From, Msg, State);
await_condition(EventType, {local_call, Msg}, State) ->
await_condition(EventType, Msg, State);
await_condition({call, _From} = EventType, {local_query, QueryFun}, State) ->
await_condition(EventType, {local_query, QueryFun, #{}}, State);
await_condition({call, From}, {local_query, QueryFun, Options}, State) ->
perform_or_delay_local_query(
await_condition, From, QueryFun, Options, State);
await_condition({call, From}, {state_query, Spec}, State) ->
Reply = {ok, do_state_query(Spec, State), id(State)},
{keep_state, State, [{reply, From, Reply}]};
await_condition(EventType, {aux_command, Cmd}, State0) ->
{_, ServerState, Effects} = ra_server:handle_aux(?FUNCTION_NAME, EventType,
Cmd, State0#state.server_state),
{State, Actions} =
?HANDLE_EFFECTS(Effects, EventType,
State0#state{server_state = ServerState}),
{keep_state, State#state{server_state = ServerState}, Actions};
await_condition({call, From}, ping, State) ->
{keep_state, State, [{reply, From, {pong, await_condition}}]};
await_condition({call, From}, trigger_election, State) ->
{keep_state, State, [{reply, From, ok},
{next_event, cast, election_timeout}]};
await_condition(info, {'DOWN', MRef, process, _Pid, _Info},
State = #state{leader_monitor = MRef}) ->
?INFO("~ts: await_condition - Leader monitor down. Entering follower state.",
[log_id(State)]),
next_state(follower, State#state{leader_monitor = undefined}, []);
await_condition(info, {'DOWN', _MRef, process, Pid, Info}, State0) ->
handle_process_down(Pid, Info, ?FUNCTION_NAME, State0);