Skip to content

Commit 87b04c3

Browse files
Merge pull request #369 from rabbitmq/gh-368
Fix snapshot installation CRC failure
2 parents 2120b01 + 7d2cd01 commit 87b04c3

11 files changed

+314
-111
lines changed

src/ra.hrl

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,10 @@
3939
%% after node restart). Pids are not stable in this sense.
4040
-type ra_server_id() :: {Name :: atom(), Node :: node()}.
4141

42-
-type ra_peer_status() :: normal | {sending_snapshot, pid()} | suspended.
42+
-type ra_peer_status() :: normal |
43+
{sending_snapshot, pid()} |
44+
suspended |
45+
disconnected.
4346

4447
-type ra_peer_state() :: #{next_index := non_neg_integer(),
4548
match_index := non_neg_integer(),

src/ra_log_snapshot.erl

Lines changed: 44 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -15,11 +15,12 @@
1515
begin_accept/2,
1616
accept_chunk/2,
1717
complete_accept/2,
18-
begin_read/1,
18+
begin_read/2,
1919
read_chunk/3,
2020
recover/1,
2121
validate/1,
22-
read_meta/1
22+
read_meta/1,
23+
context/0
2324
]).
2425

2526
-define(MAGIC, "RASN").
@@ -68,34 +69,51 @@ begin_accept(SnapDir, Meta) ->
6869
Data]),
6970
{ok, {PartialCrc, Fd}}.
7071

72+
accept_chunk(<<?MAGIC, ?VERSION:8/unsigned, Crc:32/integer,
73+
Rest/binary>> = Chunk, {_PartialCrc, Fd}) ->
74+
% ensure we overwrite the existing header when we are receiving the
75+
% full file
76+
PartialCrc = erlang:crc32(Rest),
77+
{ok, 0} = file:position(Fd, 0),
78+
ok = file:write(Fd, Chunk),
79+
{ok, {PartialCrc, Crc, Fd}};
7180
accept_chunk(Chunk, {PartialCrc, Fd}) ->
72-
<<Crc:32/integer, Rest/binary>> = Chunk,
73-
accept_chunk(Rest, {PartialCrc, Crc, Fd});
81+
%% compatibility clause where we did not receive the full file
82+
%% do not validate Crc due to OTP 26 map key ordering changes
83+
<<_Crc:32/integer, Rest/binary>> = Chunk,
84+
accept_chunk(Rest, {PartialCrc, undefined, Fd});
7485
accept_chunk(Chunk, {PartialCrc0, Crc, Fd}) ->
7586
ok = file:write(Fd, Chunk),
7687
PartialCrc = erlang:crc32(PartialCrc0, Chunk),
7788
{ok, {PartialCrc, Crc, Fd}}.
7889

79-
complete_accept(Chunk, {PartialCrc, Fd}) ->
80-
<<Crc:32/integer, Rest/binary>> = Chunk,
81-
complete_accept(Rest, {PartialCrc, Crc, Fd});
82-
complete_accept(Chunk, {PartialCrc0, Crc, Fd}) ->
83-
ok = file:write(Fd, Chunk),
84-
ok = file:pwrite(Fd, 5, <<Crc:32/integer>>),
85-
Crc = erlang:crc32(PartialCrc0, Chunk),
90+
complete_accept(Chunk, St0) ->
91+
{ok, {CalculatedCrc, Crc, Fd}} = accept_chunk(Chunk, St0),
92+
CrcToWrite = case Crc of
93+
undefined ->
94+
CalculatedCrc;
95+
_ ->
96+
Crc
97+
end,
98+
ok = file:pwrite(Fd, 5, <<CrcToWrite:32/integer>>),
8699
ok = file:sync(Fd),
87100
ok = file:close(Fd),
101+
CalculatedCrc = CrcToWrite,
88102
ok.
89103

90-
begin_read(Dir) ->
104+
begin_read(Dir, Context) ->
91105
File = filename(Dir),
92106
case file:open(File, [read, binary, raw]) of
93107
{ok, Fd} ->
94108
case read_meta_internal(Fd) of
109+
{ok, Meta, _Crc}
110+
when map_get(can_accept_full_file, Context) ->
111+
{ok, Eof} = file:position(Fd, eof),
112+
{ok, Meta, {0, Eof, Fd}};
95113
{ok, Meta, Crc} ->
96-
{ok, DataStart} = file:position(Fd, cur),
114+
{ok, Cur} = file:position(Fd, cur),
97115
{ok, Eof} = file:position(Fd, eof),
98-
{ok, Meta, {Crc, {DataStart, Eof, Fd}}};
116+
{ok, Meta, {Crc, {Cur, Eof, Fd}}};
99117
{error, _} = Err ->
100118
_ = file:close(Fd),
101119
Err
@@ -105,6 +123,7 @@ begin_read(Dir) ->
105123
end.
106124

107125
read_chunk({Crc, ReadState}, Size, Dir) when is_integer(Crc) ->
126+
%% this the compatibility read mode for old snapshot receivers
108127
case read_chunk(ReadState, Size - 4, Dir) of
109128
{ok, Data, ReadState1} ->
110129
{ok, <<Crc:32/integer, Data/binary>>, ReadState1};
@@ -156,25 +175,28 @@ validate(Dir) ->
156175
%% entire binary body. NB: this does not do checksum validation.
157176
-spec read_meta(file:filename()) ->
158177
{ok, meta()} | {error, invalid_format |
159-
{invalid_version, integer()} |
160-
checksum_error |
161-
file_err()}.
178+
{invalid_version, integer()} |
179+
checksum_error |
180+
file_err()}.
162181
read_meta(Dir) ->
163182
File = filename(Dir),
164183
case file:open(File, [read, binary, raw]) of
165184
{ok, Fd} ->
166185
case read_meta_internal(Fd) of
167-
{ok, Meta, _} ->
186+
{ok, Meta, _Crc} ->
168187
_ = file:close(Fd),
169188
{ok, Meta};
170-
{error, _} = Err ->
189+
Err ->
171190
_ = file:close(Fd),
172191
Err
173-
end;
174-
Err ->
175-
Err
192+
end
176193
end.
177194

195+
-spec context() -> map().
196+
context() ->
197+
#{can_accept_full_file => true}.
198+
199+
178200
%% Internal
179201

180202
read_meta_internal(Fd) ->

src/ra_server.erl

Lines changed: 19 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,7 @@
4949
persist_last_applied/1,
5050
update_peer/3,
5151
register_external_log_reader/2,
52+
update_disconnected_peers/3,
5253
handle_down/5,
5354
handle_node_status/6,
5455
terminate/2,
@@ -1648,20 +1649,14 @@ make_pipelined_rpc_effects(#{cfg := #cfg{id = Id,
16481649
%% TODO: refactor this please, why does make_rpc_effect need to take the
16491650
%% full state
16501651
maps:fold(
1651-
fun (I, _, Acc) when I =:= Id ->
1652-
%% oneself
1653-
Acc;
1654-
(_, #{status := suspended}, Acc) ->
1655-
Acc;
1656-
(_, #{status := {sending_snapshot, _}}, Acc) ->
1657-
%% if a peer is currently receiving a snapshot
1658-
%% do not send any append entries rpcs
1659-
Acc;
1660-
(PeerId, #{next_index := NextIdx,
1652+
fun (PeerId, #{next_index := NextIdx,
1653+
status := normal,
16611654
commit_index_sent := CI,
16621655
match_index := MatchIdx} = Peer0,
16631656
{S0, More0, Effs} = Acc)
1664-
when NextIdx < NextLogIdx orelse CI < CommitIndex ->
1657+
when PeerId =/= Id andalso
1658+
(NextIdx < NextLogIdx orelse CI < CommitIndex) ->
1659+
% the status is normal and
16651660
% there are unsent items or a new commit index
16661661
% check if the match index isn't too far behind the
16671662
% next index
@@ -1840,6 +1835,19 @@ register_external_log_reader(Pid, #{log := Log0} = State) ->
18401835
{Log, Effs} = ra_log:register_reader(Pid, Log0),
18411836
{State#{log => Log}, Effs}.
18421837

1838+
-spec update_disconnected_peers(node(), nodeup | nodedown, ra_server_state()) ->
1839+
ra_server_state().
1840+
update_disconnected_peers(Node, nodeup, #{cluster := Peers} = State) ->
1841+
State#{cluster => maps:map(
1842+
fun ({_, PeerNode}, #{status := disconnected} = Peer)
1843+
when PeerNode == Node ->
1844+
Peer#{status => normal};
1845+
(_, Peer) ->
1846+
Peer
1847+
end, Peers)};
1848+
update_disconnected_peers(_Node, _Status, State) ->
1849+
State.
1850+
18431851
peer_snapshot_process_exited(SnapshotPid, #{cluster := Peers} = State) ->
18441852
PeerKv =
18451853
maps:to_list(

src/ra_server_proc.erl

Lines changed: 44 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -1235,40 +1235,49 @@ handle_effect(_, {reply, Reply}, {call, From}, State, Actions) ->
12351235
{State, Actions};
12361236
handle_effect(_, {reply, Reply}, EvtType, _, _) ->
12371237
exit({undefined_reply, Reply, EvtType});
1238-
handle_effect(leader, {send_snapshot, To, {SnapState, Id, Term}}, _,
1238+
handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, Id, Term}}, _,
12391239
#state{server_state = SS0,
12401240
monitors = Monitors,
12411241
conf = #conf{snapshot_chunk_size = ChunkSize,
12421242
install_snap_rpc_timeout = InstallSnapTimeout} = Conf} = State0,
12431243
Actions) ->
1244-
ok = incr_counter(Conf, ?C_RA_SRV_SNAPSHOTS_SENT, 1),
1245-
%% leader effect only
1246-
Self = self(),
1247-
Machine = ra_server:machine(SS0),
1248-
Pid = spawn(fun () ->
1249-
try send_snapshots(Self, Id, Term, To,
1250-
ChunkSize, InstallSnapTimeout,
1251-
SnapState, Machine) of
1252-
_ -> ok
1253-
catch
1254-
C:timeout:S ->
1255-
%% timeout is ok as we've already blocked
1256-
%% for a while
1257-
erlang:raise(C, timeout, S);
1258-
C:E:S ->
1259-
%% insert an arbitrary pause here as a primitive
1260-
%% throttling operation as certain errors
1261-
%% happen quickly
1262-
ok = timer:sleep(5000),
1263-
erlang:raise(C, E, S)
1264-
end
1265-
end),
1266-
%% update the peer state so that no pipelined entries are sent during
1267-
%% the snapshot sending phase
1268-
SS = ra_server:update_peer(To, #{status => {sending_snapshot, Pid}}, SS0),
1269-
{State0#state{server_state = SS,
1270-
monitors = ra_monitors:add(Pid, snapshot_sender, Monitors)},
1271-
Actions};
1244+
case lists:member(ToNode, [node() | nodes()]) of
1245+
true ->
1246+
%% node is connected
1247+
%% leader effect only
1248+
Self = self(),
1249+
Machine = ra_server:machine(SS0),
1250+
Pid = spawn(fun () ->
1251+
try send_snapshots(Self, Id, Term, To,
1252+
ChunkSize, InstallSnapTimeout,
1253+
SnapState, Machine) of
1254+
_ -> ok
1255+
catch
1256+
C:timeout:S ->
1257+
%% timeout is ok as we've already blocked
1258+
%% for a while
1259+
erlang:raise(C, timeout, S);
1260+
C:E:S ->
1261+
%% insert an arbitrary pause here as a primitive
1262+
%% throttling operation as certain errors
1263+
%% happen quickly
1264+
ok = timer:sleep(5000),
1265+
erlang:raise(C, E, S)
1266+
end
1267+
end),
1268+
ok = incr_counter(Conf, ?C_RA_SRV_SNAPSHOTS_SENT, 1),
1269+
%% update the peer state so that no pipelined entries are sent during
1270+
%% the snapshot sending phase
1271+
SS = ra_server:update_peer(To, #{status => {sending_snapshot, Pid}}, SS0),
1272+
{State0#state{server_state = SS,
1273+
monitors = ra_monitors:add(Pid, snapshot_sender, Monitors)},
1274+
Actions};
1275+
false ->
1276+
?DEBUG("~s: send_snapshot node ~s disconnected",
1277+
[log_id(State0), ToNode]),
1278+
SS = ra_server:update_peer(To, #{status => disconnected}, SS0),
1279+
{State0#state{server_state = SS}, Actions}
1280+
end;
12721281
handle_effect(_, {delete_snapshot, Dir, SnapshotRef}, _, State0, Actions) ->
12731282
%% delete snapshots in separate process
12741283
_ = spawn(fun() ->
@@ -1571,11 +1580,12 @@ fold_log(From, Fun, Term, State) ->
15711580

15721581
send_snapshots(Me, Id, Term, {_, ToNode} = To, ChunkSize,
15731582
InstallTimeout, SnapState, Machine) ->
1583+
Context = ra_snapshot:context(SnapState, ToNode),
15741584
{ok, #{machine_version := SnapMacVer} = Meta, ReadState} =
1575-
ra_snapshot:begin_read(SnapState),
1585+
ra_snapshot:begin_read(SnapState, Context),
15761586

15771587
%% only send the snapshot if the target server can accept it
1578-
TheirMacVer = rpc:call(ToNode, ra_machine, version, [Machine]),
1588+
TheirMacVer = erpc:call(ToNode, ra_machine, version, [Machine]),
15791589

15801590
case SnapMacVer > TheirMacVer of
15811591
true ->
@@ -1726,14 +1736,16 @@ handle_node_status_change(Node, Status, InfoList, RaftState,
17261736
#state{monitors = Monitors0,
17271737
server_state = ServerState0} = State0) ->
17281738
{Comps, Monitors} = ra_monitors:handle_down(Node, Monitors0),
1729-
{_, ServerState, Effects} =
1739+
{_, ServerState1, Effects} =
17301740
lists:foldl(
17311741
fun (Comp, {R, S0, E0}) ->
17321742
{R, S, E} = ra_server:handle_node_status(R, Comp, Node,
17331743
Status, InfoList,
17341744
S0),
17351745
{R, S, E0 ++ E}
17361746
end, {RaftState, ServerState0, []}, Comps),
1747+
ServerState = ra_server:update_disconnected_peers(Node, Status,
1748+
ServerState1),
17371749
{State, Actions} = handle_effects(RaftState, Effects, cast,
17381750
State0#state{server_state = ServerState,
17391751
monitors = Monitors}),

src/ra_snapshot.erl

Lines changed: 26 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,7 @@
1616
-export([
1717
recover/1,
1818
read_meta/2,
19-
begin_read/1,
19+
begin_read/2,
2020
read_chunk/3,
2121
delete/2,
2222

@@ -36,6 +36,8 @@
3636
accept_chunk/4,
3737
abort_accept/1,
3838

39+
context/2,
40+
3941
handle_down/3,
4042
current_snapshot_dir/1
4143
]).
@@ -68,6 +70,8 @@
6870

6971
-export_type([state/0]).
7072

73+
-optional_callbacks([context/0]).
74+
7175
%% Side effect function
7276
%% Turn the current state into immutable reference.
7377
-callback prepare(Index :: ra_index(),
@@ -87,7 +91,9 @@
8791

8892
%% Read the snapshot metadata and initialise a read state used in read_chunk/1
8993
%% The read state should contain all the information required to read a chunk
90-
-callback begin_read(Location :: file:filename()) ->
94+
%% The Context is the map returned by the context/0 callback
95+
%% This can be used to inform the sender of receive capabilities.
96+
-callback begin_read(Location :: file:filename(), Context :: map()) ->
9197
{ok, Meta :: meta(), ReadState :: term()}
9298
| {error, term()}.
9399

@@ -131,6 +137,8 @@
131137
file_err() |
132138
term()}.
133139

140+
-callback context() -> map().
141+
134142
-spec init(ra_uid(), module(), file:filename()) ->
135143
state().
136144
init(UId, Mod, File) ->
@@ -310,6 +318,18 @@ abort_accept(#?MODULE{accepting = #accept{idxterm = {Idx, Term}},
310318
ok = delete(Dir, {Idx, Term}),
311319
State#?MODULE{accepting = undefined}.
312320

321+
%% get the snapshot capabilities context of a remote node
322+
-spec context(state(), node()) -> map().
323+
context(#?MODULE{module = Mod}, Node) ->
324+
try erpc:call(Node, Mod, ?FUNCTION_NAME, []) of
325+
Result ->
326+
Result
327+
catch
328+
error:{exception, undef, _} ->
329+
#{}
330+
end.
331+
332+
313333

314334
-spec handle_down(pid(), Info :: term(), state()) ->
315335
state().
@@ -335,14 +355,15 @@ delete(Dir, {Idx, Term}) ->
335355
ok = ra_lib:recursive_delete(SnapDir),
336356
ok.
337357

338-
-spec begin_read(State :: state()) ->
358+
-spec begin_read(State :: state(), Context :: map()) ->
339359
{ok, Meta :: meta(), ReadState} |
340360
{error, term()} when ReadState :: term().
341361
begin_read(#?MODULE{module = Mod,
342362
directory = Dir,
343-
current = {Idx, Term}}) ->
363+
current = {Idx, Term}},
364+
Context) when is_map(Context) ->
344365
Location = make_snapshot_dir(Dir, Idx, Term),
345-
Mod:begin_read(Location).
366+
Mod:begin_read(Location, Context).
346367

347368

348369
-spec read_chunk(ReadState, ChunkSizeBytes :: non_neg_integer(),

0 commit comments

Comments
 (0)