Skip to content

Commit b15eb0f

Browse files
committed
rabbit_db: join/2 now takes care of stopping/starting RabbitMQ
[Why] Up until now, a user had to run the following three commands to expand a cluster: 1. stop_app 2. join_cluster 3. start_app Stopping and starting the `rabbit` application and taking care of the underlying Mnesia application could be handled by `join_cluster` directly. [How] After the call to `can_join/1` and before proceeding with the actual join, the code remembers the state of `rabbit`, the Feature flags controler and Mnesia. After the join, it restarts whatever needs to be restarted to. It does so regardless of the success or failure of the join. One exception is when the node switched from Mnesia to Khepri as part of that join. In this case, Mnesia is left stopped.
1 parent 6bce645 commit b15eb0f

File tree

4 files changed

+78
-14
lines changed

4 files changed

+78
-14
lines changed

deps/rabbit/src/rabbit_db_cluster.erl

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -96,6 +96,28 @@ join(RemoteNode, NodeType)
9696
when is_atom(RemoteNode) andalso ?IS_NODE_TYPE(NodeType) ->
9797
case can_join(RemoteNode) of
9898
{ok, ClusterNodes} when is_list(ClusterNodes) ->
99+
%% RabbitMQ and Mnesia must be stopped to modify the cluster. In
100+
%% particular, we stop Mnesia regardless of the remotely selected
101+
%% database because we might change it during the join.
102+
RestartMnesia = rabbit_mnesia:is_running(),
103+
RestartFFCtl = rabbit_ff_controller:is_running(),
104+
RestartRabbit = rabbit:is_running(),
105+
case RestartRabbit of
106+
true ->
107+
rabbit:stop();
108+
false ->
109+
case RestartFFCtl of
110+
true ->
111+
ok = rabbit_ff_controller:wait_for_task_and_stop();
112+
false ->
113+
ok
114+
end,
115+
case RestartMnesia of
116+
true -> rabbit_mnesia:stop_mnesia();
117+
false -> ok
118+
end
119+
end,
120+
99121
ok = rabbit_db:reset(),
100122
rabbit_feature_flags:copy_feature_states_after_reset(RemoteNode),
101123

@@ -106,6 +128,27 @@ join(RemoteNode, NodeType)
106128
true -> join_using_khepri(ClusterNodes, NodeType);
107129
false -> join_using_mnesia(ClusterNodes, NodeType)
108130
end,
131+
132+
%% Restart RabbitMQ afterwards, if it was running before the join.
133+
%% Likewise for the Feature flags controller and Mnesia (if we
134+
%% still need it).
135+
case RestartRabbit of
136+
true ->
137+
rabbit:start();
138+
false ->
139+
case RestartFFCtl of
140+
true ->
141+
ok = rabbit_sup:start_child(rabbit_ff_controller);
142+
false ->
143+
ok
144+
end,
145+
NeedMnesia = not rabbit_khepri:is_enabled(),
146+
case RestartMnesia andalso NeedMnesia of
147+
true -> rabbit_mnesia:start_mnesia(false);
148+
false -> ok
149+
end
150+
end,
151+
109152
case Ret of
110153
ok ->
111154
rabbit_node_monitor:notify_joined_cluster(),

deps/rabbit/src/rabbit_mnesia.erl

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -177,7 +177,6 @@ create_cluster_callback(RemoteNode, NodeType) ->
177177
-> {ok, [node()]} | {ok, already_member} | {error, {inconsistent_cluster, string()}}.
178178

179179
can_join_cluster(DiscoveryNode) ->
180-
ensure_mnesia_not_running(),
181180
ensure_mnesia_dir(),
182181
case is_only_clustered_disc_node() of
183182
true -> e(clustering_only_disc_node);
@@ -225,6 +224,7 @@ join_cluster(ClusterNodes, NodeType) when is_list(ClusterNodes) ->
225224
join_cluster(DiscoveryNode, NodeType) when is_atom(DiscoveryNode) ->
226225
%% Code to remain compatible with `change_cluster_node_type/1' and older
227226
%% CLI.
227+
ensure_mnesia_not_running(),
228228
case can_join_cluster(DiscoveryNode) of
229229
{ok, ClusterNodes} when is_list(ClusterNodes) ->
230230
ok = reset_gracefully(),

deps/rabbit/test/clustering_management_SUITE.erl

Lines changed: 34 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -269,7 +269,30 @@ join_and_part_cluster(Config) ->
269269

270270
stop_reset_start(Config, Hare),
271271
assert_not_clustered(Hare),
272-
assert_not_clustered(Bunny).
272+
assert_not_clustered(Bunny),
273+
274+
%% Using `join_cluster` is allowed without stopping `rabbit` first since
275+
%% 3.13.0. It will only work if all nodes support it: check if they all
276+
%% expose `rabbit_ff_controller:is_running/0`.
277+
Supported = lists:all(
278+
fun(R) -> R end,
279+
rabbit_ct_broker_helpers:rpc_all(
280+
Config, erlang, function_exported,
281+
[rabbit_ff_controller, is_running, 0])),
282+
ct:pal(
283+
"Do all nodes support `join_cluster` without stopping `rabbit` "
284+
"first? ~p",
285+
[Supported]),
286+
case Supported of
287+
true ->
288+
?assertEqual(ok, join_cluster(Config, Rabbit, Bunny)),
289+
assert_clustered([Rabbit, Bunny]),
290+
291+
?assertEqual(ok, join_cluster(Config, Hare, Bunny)),
292+
assert_clustered([Rabbit, Bunny, Hare]);
293+
false ->
294+
ok
295+
end.
273296

274297
stop_start_cluster_node(Config) ->
275298
[Rabbit, Hare] = cluster_members(Config),
@@ -326,7 +349,15 @@ join_and_part_cluster_in_khepri(Config) ->
326349

327350
stop_reset_start(Config, Rabbit),
328351
assert_not_clustered(Rabbit),
329-
assert_not_clustered(Hare).
352+
assert_not_clustered(Hare),
353+
354+
%% Using `join_cluster` is allowed without stopping `rabbit` first since
355+
%% 3.13.0.
356+
?assertEqual(ok, join_cluster(Config, Rabbit, Bunny)),
357+
assert_clustered([Rabbit, Bunny]),
358+
359+
?assertEqual(ok, join_cluster(Config, Hare, Bunny)),
360+
assert_clustered([Rabbit, Bunny, Hare]).
330361

331362
join_cluster_bad_operations(Config) ->
332363
[Rabbit, Hare, Bunny] = cluster_members(Config),
@@ -342,10 +373,6 @@ join_cluster_bad_operations(Config) ->
342373
ok = start_app(Config, Rabbit),
343374
assert_not_clustered(Rabbit),
344375

345-
%% Trying to cluster with mnesia running
346-
assert_failure(fun () -> join_cluster(Config, Rabbit, Bunny) end),
347-
assert_not_clustered(Rabbit),
348-
349376
%% Trying to cluster the node with itself
350377
ok = stop_app(Config, Rabbit),
351378
assert_failure(fun () -> join_cluster(Config, Rabbit, Rabbit) end),
@@ -379,18 +406,14 @@ join_cluster_bad_operations(Config) ->
379406
ok.
380407

381408
join_cluster_bad_operations_in_khepri(Config) ->
382-
[Rabbit, _Hare, Bunny] = cluster_members(Config),
409+
[Rabbit, _Hare, _Bunny] = cluster_members(Config),
383410

384411
%% Nonexistent node
385412
ok = stop_app(Config, Rabbit),
386413
assert_failure(fun () -> join_cluster(Config, Rabbit, non@existent) end),
387414
ok = start_app(Config, Rabbit),
388415
assert_not_clustered(Rabbit),
389416

390-
%% Trying to cluster with mnesia running
391-
assert_failure(fun () -> join_cluster(Config, Rabbit, Bunny) end),
392-
assert_not_clustered(Rabbit),
393-
394417
%% Trying to cluster the node with itself
395418
ok = stop_app(Config, Rabbit),
396419
assert_failure(fun () -> join_cluster(Config, Rabbit, Rabbit) end),

deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/join_cluster_command.ex

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -28,8 +28,6 @@ defmodule RabbitMQ.CLI.Ctl.Commands.JoinClusterCommand do
2828
def validate([_], _), do: :ok
2929
def validate(_, _), do: {:validation_failure, :too_many_args}
3030

31-
use RabbitMQ.CLI.Core.RequiresRabbitAppStopped
32-
3331
def run([target_node], %{node: node_name, ram: ram, disc: disc} = opts) do
3432
node_type =
3533
case {ram, disc} do

0 commit comments

Comments
 (0)