Skip to content

Commit fcae963

Browse files
dumbbellmergify[bot]
authored andcommitted
rabbit_peer_discovery: Compute start time once
... and cache it. [Why] It happens at least in CI that the computed start time varies by a few seconds. I think this comes from the Erlang time offset which might be adjusted over time. This affects peer discovery's sorting of RabbitMQ nodes which uses that start time to determine the oldest node. When the start time of a node changes, it could be considered the seed node to join by some nodes but ignored by the other nodes, leading to troubles with cluster formation. (cherry picked from commit e72d911)
1 parent 880ca8f commit fcae963

File tree

3 files changed

+75
-14
lines changed

3 files changed

+75
-14
lines changed

deps/rabbit/src/rabbit_peer_discovery.erl

Lines changed: 17 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -627,8 +627,7 @@ query_node_props2([{Node, Members} | Rest], NodesAndProps, FromNode) ->
627627
["Peer discovery: temporary hidden node '~ts' "
628628
"queries properties from node '~ts'",
629629
[node(), Node]], FromNode),
630-
StartTime = get_node_start_time(
631-
Node, microsecond, FromNode),
630+
StartTime = get_node_start_time(Node, FromNode),
632631
IsReady = is_node_db_ready(Node, FromNode),
633632
NodeAndProps = {Node, Members, StartTime, IsReady},
634633
NodesAndProps1 = [NodeAndProps | NodesAndProps],
@@ -656,9 +655,8 @@ query_node_props2([], NodesAndProps, _FromNode) ->
656655
?assert(length(NodesAndProps1) =< length(nodes(hidden))),
657656
NodesAndProps1.
658657

659-
-spec get_node_start_time(Node, Unit, FromNode) -> StartTime when
658+
-spec get_node_start_time(Node, FromNode) -> StartTime when
660659
Node :: node(),
661-
Unit :: erlang:time_unit(),
662660
FromNode :: node(),
663661
StartTime :: non_neg_integer().
664662
%% @doc Returns the start time of the given `Node' in `Unit'.
@@ -679,15 +677,21 @@ query_node_props2([], NodesAndProps, _FromNode) ->
679677
%%
680678
%% @private
681679

682-
get_node_start_time(Node, Unit, FromNode) ->
683-
NativeStartTime = erpc_call(
684-
Node, erlang, system_info, [start_time], FromNode),
685-
TimeOffset = erpc_call(Node, erlang, time_offset, [], FromNode),
686-
SystemStartTime = NativeStartTime + TimeOffset,
687-
StartTime = erpc_call(
688-
Node, erlang, convert_time_unit,
689-
[SystemStartTime, native, Unit], FromNode),
690-
StartTime.
680+
get_node_start_time(Node, FromNode) ->
681+
try
682+
erpc_call(Node,rabbit_boot_state, get_start_time, [], FromNode)
683+
catch
684+
error:{exception, _, _} ->
685+
NativeStartTime = erpc_call(
686+
Node, erlang, system_info, [start_time],
687+
FromNode),
688+
TimeOffset = erpc_call(Node, erlang, time_offset, [], FromNode),
689+
SystemStartTime = NativeStartTime + TimeOffset,
690+
StartTime = erpc_call(
691+
Node, erlang, convert_time_unit,
692+
[SystemStartTime, native, microsecond], FromNode),
693+
StartTime
694+
end.
691695

692696
-spec is_node_db_ready(Node, FromNode) -> IsReady when
693697
Node :: node(),

deps/rabbitmq_prelaunch/src/rabbit_boot_state.erl

Lines changed: 57 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,12 @@
1717
set/1,
1818
wait_for/2,
1919
has_reached/1,
20-
has_reached_and_is_active/1]).
20+
has_reached_and_is_active/1,
21+
get_start_time/0,
22+
record_start_time/0]).
2123

2224
-define(PT_KEY_BOOT_STATE, {?MODULE, boot_state}).
25+
-define(PT_KEY_START_TIME, {?MODULE, start_time}).
2326

2427
-type boot_state() :: stopped |
2528
booting |
@@ -95,3 +98,56 @@ has_reached_and_is_active(TargetBootState) ->
9598
andalso
9699
not has_reached(CurrentBootState, stopping)
97100
end.
101+
102+
-spec get_start_time() -> StartTime when
103+
StartTime :: integer().
104+
%% @doc Returns the start time of the Erlang VM.
105+
%%
106+
%% This time was recorded by {@link record_start_time/0} as early as possible
107+
%% and is immutable.
108+
109+
get_start_time() ->
110+
persistent_term:get(?PT_KEY_START_TIME).
111+
112+
-spec record_start_time() -> ok.
113+
%% @doc Records the start time of the Erlang VM.
114+
%%
115+
%% The time is expressed in microseconds since Epoch. It can be compared to
116+
%% other non-native times. This is used by the Peer Discovery subsystem to
117+
%% sort nodes and select a seed node if the peer discovery backend did not
118+
%% select one.
119+
%%
120+
%% This time is recorded once. Calling this function multiple times won't
121+
%% overwrite the value.
122+
123+
record_start_time() ->
124+
Key = ?PT_KEY_START_TIME,
125+
try
126+
%% Check if the start time was recorded.
127+
_ = persistent_term:get(Key),
128+
ok
129+
catch
130+
error:badarg ->
131+
%% The start time was not recorded yet. Acquire a lock and check
132+
%% again in case another process got the lock first and recorded
133+
%% the start time.
134+
Node = node(),
135+
LockId = {?PT_KEY_START_TIME, self()},
136+
true = global:set_lock(LockId, [Node]),
137+
try
138+
_ = persistent_term:get(Key),
139+
ok
140+
catch
141+
error:badarg ->
142+
%% We are really the first to get the lock and we can
143+
%% record the start time.
144+
NativeStartTime = erlang:system_info(start_time),
145+
TimeOffset = erlang:time_offset(),
146+
SystemStartTime = NativeStartTime + TimeOffset,
147+
StartTime = erlang:convert_time_unit(
148+
SystemStartTime, native, microsecond),
149+
persistent_term:put(Key, StartTime)
150+
after
151+
global:del_lock(LockId, [Node])
152+
end
153+
end.

deps/rabbitmq_prelaunch/src/rabbit_prelaunch.erl

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,7 @@ run_prelaunch_first_phase() ->
4848
do_run() ->
4949
%% Indicate RabbitMQ is booting.
5050
clear_stop_reason(),
51+
rabbit_boot_state:record_start_time(),
5152
rabbit_boot_state:set(booting),
5253

5354
%% Configure dbg if requested.

0 commit comments

Comments
 (0)