Skip to content

Commit eb6327a

Browse files
committed
rabbit_db: Move generic init steps from rabbit_mnesia
[Why] When a single node or a cluster is initialized, we go through a few steps which are not Mnesia-specific or even related. Things such as synchronizing feature flag states or emit a "cluster joined" notification with `rabbit_node_monitor`. When we will introduce Khepri, we will have to go through the same generic steps. Therefore, it makes sense to drive those steps from `rabbit_db` and `rabbit_db_cluster` and only call into `rabbit_mnesia` when needed. [How] The generic code is moved from `rabbit_mnesia` to `rabbit_db*` and `rabbit_peer_discovery` modules.
1 parent 9c358dd commit eb6327a

File tree

4 files changed

+75
-47
lines changed

4 files changed

+75
-47
lines changed

deps/rabbit/src/rabbit_db.erl

Lines changed: 35 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -42,12 +42,23 @@ init() ->
4242
?LOG_DEBUG(
4343
"DB: this node is virgin: ~ts", [IsVirgin],
4444
#{domain => ?RMQLOG_DOMAIN_DB}),
45+
4546
ensure_dir_exists(),
46-
case init_using_mnesia() of
47+
rabbit_peer_discovery:log_configured_backend(),
48+
rabbit_peer_discovery:maybe_init(),
49+
50+
pre_init(IsVirgin),
51+
52+
Ret = run(
53+
#{mnesia => fun init_using_mnesia/0}),
54+
case Ret of
4755
ok ->
4856
?LOG_DEBUG(
4957
"DB: initialization successeful",
5058
#{domain => ?RMQLOG_DOMAIN_DB}),
59+
60+
post_init(IsVirgin),
61+
5162
ok;
5263
Error ->
5364
?LOG_DEBUG(
@@ -56,6 +67,17 @@ init() ->
5667
Error
5768
end.
5869

70+
pre_init(IsVirgin) ->
71+
Members = rabbit_db_cluster:members(),
72+
OtherMembers = rabbit_nodes:nodes_excl_me(Members),
73+
rabbit_db_cluster:ensure_feature_flags_are_in_sync(OtherMembers, IsVirgin).
74+
75+
post_init(false = _IsVirgin) ->
76+
rabbit_peer_discovery:maybe_register();
77+
post_init(true = _IsVirgin) ->
78+
%% Registration handled by rabbit_peer_discovery.
79+
ok.
80+
5981
init_using_mnesia() ->
6082
?LOG_DEBUG(
6183
"DB: initialize Mnesia",
@@ -115,8 +137,11 @@ force_load_on_next_boot_using_mnesia() ->
115137
%% @see is_virgin_node/1.
116138

117139
is_virgin_node() ->
118-
ThisNode = node(),
119-
is_virgin_node(ThisNode).
140+
run(
141+
#{mnesia => fun is_virgin_node_using_mnesia/0}).
142+
143+
is_virgin_node_using_mnesia() ->
144+
rabbit_mnesia:is_virgin_node().
120145

121146
-spec is_virgin_node(Node) -> IsVirgin | undefined when
122147
Node :: node(),
@@ -129,14 +154,11 @@ is_virgin_node() ->
129154
%% @returns `true' if the node is virgin, `false' if it is not, or `undefined'
130155
%% if the given node is remote and we couldn't determine it.
131156

157+
is_virgin_node(Node) when Node =:= node() ->
158+
is_virgin_node();
132159
is_virgin_node(Node) when is_atom(Node) ->
133-
is_virgin_node_with_mnesia(Node).
134-
135-
is_virgin_node_with_mnesia(Node) when Node =:= node() ->
136-
rabbit_mnesia:is_virgin_node();
137-
is_virgin_node_with_mnesia(Node) ->
138160
try
139-
erpc:call(Node, rabbit_mnesia, is_virgin_node, [], ?TIMEOUT)
161+
erpc:call(Node, ?MODULE, is_virgin_node, [], ?TIMEOUT)
140162
catch
141163
_:_ ->
142164
undefined
@@ -149,7 +171,8 @@ is_virgin_node_with_mnesia(Node) ->
149171
%% @returns the directory path.
150172

151173
dir() ->
152-
mnesia_dir().
174+
run(
175+
#{mnesia => fun mnesia_dir/0}).
153176

154177
mnesia_dir() ->
155178
rabbit_mnesia:dir().
@@ -183,9 +206,9 @@ ensure_dir_exists() ->
183206
run(Funs)
184207
when is_map(Funs) andalso is_map_key(mnesia, Funs) ->
185208
#{mnesia := MnesiaFun} = Funs,
186-
run_with_mnesia(MnesiaFun).
209+
run_using_mnesia(MnesiaFun).
187210

188-
run_with_mnesia(Fun) ->
211+
run_using_mnesia(Fun) ->
189212
Fun().
190213

191214
list_in_mnesia(Table, Match) ->

deps/rabbit/src/rabbit_db_cluster.erl

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -11,7 +11,8 @@
1111

1212
-include_lib("rabbit_common/include/logging.hrl").
1313

14-
-export([join/2,
14+
-export([ensure_feature_flags_are_in_sync/2,
15+
join/2,
1516
forget_member/2]).
1617
-export([change_node_type/1]).
1718
-export([is_clustered/0,
@@ -36,6 +37,14 @@
3637
%% Cluster formation.
3738
%% -------------------------------------------------------------------
3839

40+
ensure_feature_flags_are_in_sync(Nodes, NodeIsVirgin) ->
41+
Ret = rabbit_feature_flags:sync_feature_flags_with_cluster(
42+
Nodes, NodeIsVirgin),
43+
case Ret of
44+
ok -> ok;
45+
{error, Reason} -> throw({error, {incompatible_feature_flags, Reason}})
46+
end.
47+
3948
-spec join(RemoteNode, NodeType) -> Ret when
4049
RemoteNode :: node(),
4150
NodeType :: rabbit_db_cluster:node_type(),

deps/rabbit/src/rabbit_mnesia.erl

Lines changed: 1 addition & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -99,8 +99,6 @@ init() ->
9999
rabbit_log:info("Node database directory at ~ts is empty. "
100100
"Assuming we need to join an existing cluster or initialise from scratch...",
101101
[dir()]),
102-
rabbit_peer_discovery:log_configured_backend(),
103-
rabbit_peer_discovery:maybe_init(),
104102
rabbit_peer_discovery:maybe_create_cluster(
105103
fun create_cluster_callback/2);
106104
false ->
@@ -115,9 +113,7 @@ init() ->
115113
true ->
116114
init_db_and_upgrade(cluster_nodes(all), NodeType,
117115
NodeType =:= ram, _Retry = true)
118-
end,
119-
rabbit_peer_discovery:maybe_init(),
120-
rabbit_peer_discovery:maybe_register()
116+
end
121117
end,
122118
%% We intuitively expect the global name server to be synced when
123119
%% Mnesia is up. In fact that's not guaranteed to be the case -
@@ -132,7 +128,6 @@ create_cluster_callback(none, NodeType) ->
132128
true -> NodeType
133129
end,
134130
init_db_and_upgrade(DiscNodes, NodeType1, true, _Retry = true),
135-
rabbit_node_monitor:notify_joined_cluster(),
136131
ok;
137132
create_cluster_callback(RemoteNode, NodeType) ->
138133
{ok, {_, DiscNodes, _}} = discover_cluster0(RemoteNode),
@@ -141,7 +136,6 @@ create_cluster_callback(RemoteNode, NodeType) ->
141136
true -> NodeType
142137
end,
143138
init_db_and_upgrade(DiscNodes, NodeType1, true, _Retry = true),
144-
rabbit_node_monitor:notify_joined_cluster(),
145139
ok.
146140

147141
%% Make the node join a cluster. The node will be reset automatically
@@ -517,23 +511,6 @@ init_db(ClusterNodes, NodeType, CheckOtherNodes) ->
517511

518512
NodeIsVirgin = is_virgin_node(),
519513
rabbit_log:debug("Does data directory looks like that of a blank (uninitialised) node? ~tp", [NodeIsVirgin]),
520-
%% We want to synchronize feature flags first before we wait for
521-
%% tables (which is needed to ensure the local view of the tables
522-
%% matches the rest of the cluster). The reason is that some
523-
%% feature flags may add or remove tables. In this case the list
524-
%% of tables returned by `rabbit_table:definitions()' usually
525-
%% depends on the state of feature flags but this state is local.
526-
%%
527-
%% For instance, a feature flag may remove a table (so it's gone
528-
%% from the cluster). If we were to wait for that table locally
529-
%% before synchronizing feature flags, we would wait forever;
530-
%% indeed the feature flag being disabled before sync,
531-
%% `rabbit_table:definitions()' would return the old table.
532-
%%
533-
%% Feature flags need to be synced before any change to Mnesia
534-
%% membership. If enabling feature flags fails, Mnesia could remain
535-
%% in an inconsistent state that prevents later joining the nodes.
536-
ensure_feature_flags_are_in_sync(rabbit_nodes:nodes_excl_me(ClusterNodes), NodeIsVirgin),
537514
Nodes = change_extra_db_nodes(ClusterNodes, CheckOtherNodes),
538515
%% Note that we use `system_info' here and not the cluster status
539516
%% since when we start rabbit for the first time the cluster
@@ -623,14 +600,6 @@ ensure_mnesia_not_running() ->
623600
throw({error, mnesia_unexpectedly_running})
624601
end.
625602

626-
ensure_feature_flags_are_in_sync(Nodes, NodeIsVirgin) ->
627-
Ret = rabbit_feature_flags:sync_feature_flags_with_cluster(
628-
Nodes, NodeIsVirgin),
629-
case Ret of
630-
ok -> ok;
631-
{error, Reason} -> throw({error, {incompatible_feature_flags, Reason}})
632-
end.
633-
634603
ensure_schema_integrity() ->
635604
case rabbit_table:check_schema_integrity(_Retry = true) of
636605
ok ->

deps/rabbit/src/rabbit_peer_discovery.erl

Lines changed: 29 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -259,7 +259,7 @@ join_discovered_peers_with_retries(
259259
"Starting as a blank standalone node...",
260260
[string:join(lists:map(fun atom_to_list/1, TryNodes), ",")],
261261
#{domain => ?RMQLOG_DOMAIN_PEER_DISC}),
262-
CreateClusterCallback(none, disc);
262+
init_single_node(CreateClusterCallback);
263263
join_discovered_peers_with_retries(
264264
TryNodes, NodeType, RetriesLeft, DelayInterval, CreateClusterCallback) ->
265265
case find_reachable_peer_to_cluster_with(TryNodes) of
@@ -268,7 +268,7 @@ join_discovered_peers_with_retries(
268268
"Peer discovery: Node '~ts' selected for auto-clustering",
269269
[Node],
270270
#{domain => ?RMQLOG_DOMAIN_PEER_DISC}),
271-
CreateClusterCallback(Node, NodeType);
271+
create_cluster(Node, NodeType, CreateClusterCallback);
272272
none ->
273273
RetriesLeft1 = RetriesLeft - 1,
274274
?LOG_INFO(
@@ -298,6 +298,33 @@ find_reachable_peer_to_cluster_with([Node | Nodes]) when Node =/= node() ->
298298
find_reachable_peer_to_cluster_with([Node | Nodes]) when Node =:= node() ->
299299
find_reachable_peer_to_cluster_with(Nodes).
300300

301+
init_single_node(CreateClusterCallback) ->
302+
IsVirgin = rabbit_db:is_virgin_node(),
303+
rabbit_db_cluster:ensure_feature_flags_are_in_sync([], IsVirgin),
304+
CreateClusterCallback(none, disc),
305+
ok.
306+
307+
create_cluster(RemoteNode, NodeType, CreateClusterCallback) ->
308+
%% We want to synchronize feature flags first before we update the cluster
309+
%% membership. This is needed to ensure the local list of Mnesia tables
310+
%% matches the rest of the cluster for example, in case a feature flag
311+
%% adds or removes tables.
312+
%%
313+
%% For instance, a feature flag may remove a table (so it's gone from the
314+
%% cluster). If we were to wait for that table locally before
315+
%% synchronizing feature flags, we would wait forever; indeed the feature
316+
%% flag being disabled before sync, `rabbit_table:definitions()' would
317+
%% return the old table.
318+
%%
319+
%% Feature flags need to be synced before any change to Mnesia membership.
320+
%% If enabling feature flags fails, Mnesia could remain in an inconsistent
321+
%% state that prevents later joining the nodes.
322+
IsVirgin = rabbit_db:is_virgin_node(),
323+
rabbit_db_cluster:ensure_feature_flags_are_in_sync([RemoteNode], IsVirgin),
324+
CreateClusterCallback(RemoteNode, NodeType),
325+
rabbit_node_monitor:notify_joined_cluster(),
326+
ok.
327+
301328
%% This module doesn't currently sanity-check the return value of
302329
%% `Backend:list_nodes()`. Therefore, it could return something invalid:
303330
%% thus the `{œk, any()} in the spec.

0 commit comments

Comments
 (0)